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