risingwave_frontend/optimizer/rule/apply_limit_transpose_rule.rs
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use itertools::Itertools;
use risingwave_common::util::sort_util::{ColumnOrder, OrderType};
use risingwave_pb::plan_common::JoinType;
use super::{BoxedRule, Rule};
use crate::optimizer::plan_node::{
LogicalApply, LogicalFilter, LogicalLimit, LogicalTopN, PlanTreeNodeUnary,
};
use crate::optimizer::property::Order;
use crate::optimizer::PlanRef;
use crate::utils::Condition;
/// Transpose `LogicalApply` and `LogicalLimit`.
///
/// Before:
///
/// ```text
/// LogicalApply
/// / \
/// Domain LogicalLimit
/// |
/// Input
/// ```
///
/// After:
///
/// ```text
/// LogicalTopN
/// |
/// LogicalApply
/// / \
/// Domain Input
/// ```
pub struct ApplyLimitTransposeRule {}
impl Rule for ApplyLimitTransposeRule {
fn apply(&self, plan: PlanRef) -> Option<PlanRef> {
let apply: &LogicalApply = plan.as_logical_apply()?;
let (left, right, on, join_type, correlated_id, correlated_indices, max_one_row) =
apply.clone().decompose();
assert_eq!(join_type, JoinType::Inner);
let logical_limit: &LogicalLimit = right.as_logical_limit()?;
let limit_input = logical_limit.input();
let limit = logical_limit.limit();
let offset = logical_limit.offset();
let apply_left_len = left.schema().len();
if max_one_row {
return None;
}
let new_apply = LogicalApply::create(
left,
limit_input,
JoinType::Inner,
Condition::true_cond(),
correlated_id,
correlated_indices,
false,
);
let new_topn = {
// use the first column as an order to provide determinism for streaming queries.
let order = Order::new(vec![ColumnOrder::new(0, OrderType::ascending())]);
LogicalTopN::new(
new_apply,
limit,
offset,
false,
order,
(0..apply_left_len).collect_vec(),
)
};
let filter = LogicalFilter::create(new_topn.into(), on);
Some(filter)
}
}
impl ApplyLimitTransposeRule {
pub fn create() -> BoxedRule {
Box::new(ApplyLimitTransposeRule {})
}
}