risingwave_frontend/optimizer/rule/max_one_row_eliminate_rule.rs
1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7// http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use super::prelude::{PlanRef, *};
16use crate::optimizer::plan_node::{LogicalApply, LogicalLimit, LogicalMaxOneRow};
17use crate::optimizer::plan_visitor::{LogicalCardinalityExt, SoleSysTableVisitor};
18
19/// Eliminate max one row restriction from `LogicalApply`.
20///
21/// If we cannot guarantee that the right side of `Apply` will return at most one row
22/// in compile time, we will add a `MaxOneRow` that does runtime check to satisfy the
23/// max one row restriction.
24///
25/// As a result, the `max_one_row` flag of `LogicalApply` will always be `false`
26/// after applying this rule.
27pub struct MaxOneRowEliminateRule {}
28impl Rule<Logical> for MaxOneRowEliminateRule {
29 fn apply(&self, plan: PlanRef) -> Option<PlanRef> {
30 let apply: &LogicalApply = plan.as_logical_apply()?;
31 let (left, mut right, on, join_type, correlated_id, correlated_indices, max_one_row) =
32 apply.clone().decompose();
33
34 if !max_one_row {
35 return None;
36 }
37
38 if !right.max_one_row() {
39 right = if SoleSysTableVisitor::sys_table_only(right.clone()) {
40 // If the right side is just a `SysScan` (with `Values`), we add a `Limit 1` to enforce the max one row restriction.
41 // This is a workaround for the case where `SysScan` cannot be guaranteed to return at most one row in compile time,
42 // but to make the system queries work compatible with PostgreSQL, we need to enforce the max one row restriction at runtime.
43 LogicalLimit::create(right, 1, 0)
44 } else {
45 LogicalMaxOneRow::create(right)
46 };
47 debug_assert!(right.max_one_row());
48 }
49
50 Some(LogicalApply::create(
51 left,
52 right,
53 join_type,
54 on,
55 correlated_id,
56 correlated_indices,
57 false,
58 ))
59 }
60}
61
62impl MaxOneRowEliminateRule {
63 pub fn create() -> BoxedRule {
64 Box::new(MaxOneRowEliminateRule {})
65 }
66}