risingwave_frontend/optimizer/rule/
apply_dedup_transpose_rule.rs1use risingwave_pb::plan_common::JoinType;
16
17use super::{BoxedRule, Rule};
18use crate::optimizer::PlanRef;
19use crate::optimizer::plan_node::{LogicalApply, LogicalDedup, LogicalFilter, PlanTreeNodeUnary};
20use crate::utils::Condition;
21
22pub struct ApplyDedupTransposeRule {}
44impl Rule for ApplyDedupTransposeRule {
45 fn apply(&self, plan: PlanRef) -> Option<PlanRef> {
46 let apply: &LogicalApply = plan.as_logical_apply()?;
47 let (left, right, on, join_type, correlated_id, correlated_indices, max_one_row) =
48 apply.clone().decompose();
49 assert_eq!(join_type, JoinType::Inner);
50 let dedup: &LogicalDedup = right.as_logical_dedup()?;
51 let dedup_cols = dedup.dedup_cols();
52 let dedup_input = dedup.input();
53
54 let apply_left_len = left.schema().len();
55
56 if max_one_row {
57 return None;
58 }
59
60 let new_apply = LogicalApply::create(
61 left,
62 dedup_input,
63 JoinType::Inner,
64 Condition::true_cond(),
65 correlated_id,
66 correlated_indices,
67 false,
68 );
69
70 let new_dedup = {
71 let mut new_dedup_cols: Vec<usize> = (0..apply_left_len).collect();
72 new_dedup_cols.extend(dedup_cols.iter().map(|key| key + apply_left_len));
73 LogicalDedup::new(new_apply, new_dedup_cols).into()
74 };
75
76 let filter = LogicalFilter::create(new_dedup, on);
77 Some(filter)
78 }
79}
80
81impl ApplyDedupTransposeRule {
82 pub fn create() -> BoxedRule {
83 Box::new(ApplyDedupTransposeRule {})
84 }
85}