risingwave_frontend/optimizer/rule/over_window_split_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 std::collections::HashMap;
16
17use itertools::Itertools;
18
19use super::prelude::{PlanRef, *};
20
21pub struct OverWindowSplitRule;
22
23impl OverWindowSplitRule {
24 pub fn create() -> BoxedRule {
25 Box::new(OverWindowSplitRule)
26 }
27}
28
29impl Rule<Logical> for OverWindowSplitRule {
30 fn apply(&self, plan: PlanRef) -> Option<PlanRef> {
31 let over_window = plan.as_logical_over_window()?;
32 let mut rank_func_seq = 0;
33 let groups: HashMap<_, _> = over_window
34 .window_functions()
35 .iter()
36 .enumerate()
37 .map(|(idx, func)| {
38 let func_seq = if func.kind.is_numbering() {
39 rank_func_seq += 1;
40 rank_func_seq
41 } else {
42 0
43 };
44 ((&func.order_by, &func.partition_by, func_seq), idx)
45 })
46 .into_group_map();
47 Some(over_window.split_with_rule(groups.into_values().sorted().collect()))
48 }
49}