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::{BoxedRule, Rule};
16use crate::optimizer::PlanRef;
17use crate::optimizer::plan_node::{LogicalApply, LogicalMaxOneRow};
18use crate::optimizer::plan_visitor::LogicalCardinalityExt;
19
20/// Eliminate max one row restriction from `LogicalApply`.
21///
22/// If we cannot guarantee that the right side of `Apply` will return at most one row
23/// in compile time, we will add a `MaxOneRow` that does runtime check to satisfy the
24/// max one row restriction.
25///
26/// As a result, the `max_one_row` flag of `LogicalApply` will always be `false`
27/// after applying this rule.
28pub struct MaxOneRowEliminateRule {}
29impl Rule for MaxOneRowEliminateRule {
30 fn apply(&self, plan: PlanRef) -> Option<PlanRef> {
31 let apply: &LogicalApply = plan.as_logical_apply()?;
32 let (left, mut right, on, join_type, correlated_id, correlated_indices, max_one_row) =
33 apply.clone().decompose();
34
35 if !max_one_row {
36 return None;
37 }
38
39 if !right.max_one_row() {
40 right = LogicalMaxOneRow::create(right);
41 debug_assert!(right.max_one_row());
42 }
43
44 Some(LogicalApply::create(
45 left,
46 right,
47 join_type,
48 on,
49 correlated_id,
50 correlated_indices,
51 false,
52 ))
53 }
54}
55
56impl MaxOneRowEliminateRule {
57 pub fn create() -> BoxedRule {
58 Box::new(MaxOneRowEliminateRule {})
59 }
60}