risingwave_frontend/optimizer/rule/
apply_over_window_transpose_rule.rs1use risingwave_pb::plan_common::JoinType;
16
17use super::{BoxedRule, Rule};
18use crate::expr::InputRef;
19use crate::optimizer::PlanRef;
20use crate::optimizer::plan_node::{LogicalApply, LogicalFilter, LogicalOverWindow};
21use crate::utils::Condition;
22
23pub struct ApplyOverWindowTransposeRule {}
45impl Rule for ApplyOverWindowTransposeRule {
46 fn apply(&self, plan: PlanRef) -> Option<PlanRef> {
47 let apply: &LogicalApply = plan.as_logical_apply()?;
48 let (left, right, on, join_type, correlated_id, correlated_indices, max_one_row) =
49 apply.clone().decompose();
50 assert_eq!(join_type, JoinType::Inner);
51 let over_window: &LogicalOverWindow = right.as_logical_over_window()?;
52 let (window_input, mut window_functions) = over_window.clone().decompose();
53
54 if max_one_row {
55 return None;
56 }
57
58 let apply_left_len = left.schema().len();
59 let apply_left_schema = left.schema().clone();
60
61 let new_apply = LogicalApply::create(
62 left,
63 window_input,
64 JoinType::Inner,
65 Condition::true_cond(),
66 correlated_id,
67 correlated_indices,
68 false,
69 );
70
71 let new_over_window = {
72 window_functions.iter_mut().for_each(|func| {
75 func.args
76 .iter_mut()
77 .for_each(|arg| arg.shift_with_offset(apply_left_len as isize));
78 func.order_by
79 .iter_mut()
80 .for_each(|c| c.column_index += apply_left_len);
81 func.partition_by
82 .iter_mut()
83 .for_each(|x| x.shift_with_offset(apply_left_len as isize));
84 func.partition_by = (0..apply_left_len)
86 .map(|i| InputRef::new(i, apply_left_schema.fields[i].data_type()))
87 .chain(func.partition_by.drain(..))
88 .collect();
89 });
90
91 LogicalOverWindow::new(window_functions, new_apply)
92 };
93
94 let filter = LogicalFilter::create(new_over_window.into(), on);
95 Some(filter)
96 }
97}
98
99impl ApplyOverWindowTransposeRule {
100 pub fn create() -> BoxedRule {
101 Box::new(ApplyOverWindowTransposeRule {})
102 }
103}