risingwave_frontend/optimizer/plan_node/
logical_dedup.rs1use fixedbitset::FixedBitSet;
16use itertools::Itertools;
17use risingwave_common::util::column_index_mapping::ColIndexMapping;
18
19use super::generic::TopNLimit;
20use super::utils::impl_distill_by_unit;
21use super::{
22 BatchGroupTopN, ColPrunable, ColumnPruningContext, ExprRewritable, Logical, LogicalProject,
23 PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, PredicatePushdownContext,
24 RewriteStreamContext, StreamDedup, StreamGroupTopN, ToBatch, ToStream, ToStreamContext,
25 gen_filter_and_pushdown, generic,
26};
27use crate::error::Result;
28use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
29use crate::optimizer::property::{Order, RequiredDist};
30use crate::utils::Condition;
31
32#[derive(Debug, Clone, PartialEq, Eq, Hash)]
35pub struct LogicalDedup {
36 pub base: PlanBase<Logical>,
37 core: generic::Dedup<PlanRef>,
38}
39
40impl LogicalDedup {
41 pub fn new(input: PlanRef, dedup_cols: Vec<usize>) -> Self {
42 let core = generic::Dedup::new(input, dedup_cols);
43 let base = PlanBase::new_logical_with_core(&core);
44 LogicalDedup { base, core }
45 }
46
47 pub fn dedup_cols(&self) -> &[usize] {
48 &self.core.dedup_cols
49 }
50}
51
52impl PlanTreeNodeUnary for LogicalDedup {
53 fn input(&self) -> PlanRef {
54 self.core.input.clone()
55 }
56
57 fn clone_with_input(&self, input: PlanRef) -> Self {
58 Self::new(input, self.dedup_cols().to_vec())
59 }
60
61 fn rewrite_with_input(
62 &self,
63 input: PlanRef,
64 input_col_change: ColIndexMapping,
65 ) -> (Self, ColIndexMapping) {
66 (
67 Self::new(
68 input,
69 self.dedup_cols()
70 .iter()
71 .map(|idx| input_col_change.map(*idx))
72 .collect_vec(),
73 ),
74 input_col_change,
75 )
76 }
77}
78
79impl_plan_tree_node_for_unary! {LogicalDedup}
80
81impl PredicatePushdown for LogicalDedup {
82 fn predicate_pushdown(
83 &self,
84 predicate: Condition,
85 ctx: &mut PredicatePushdownContext,
86 ) -> PlanRef {
87 gen_filter_and_pushdown(self, predicate, Condition::true_cond(), ctx)
88 }
89}
90
91impl ToStream for LogicalDedup {
92 fn logical_rewrite_for_stream(
93 &self,
94 ctx: &mut RewriteStreamContext,
95 ) -> Result<(PlanRef, ColIndexMapping)> {
96 let (input, input_col_change) = self.input().logical_rewrite_for_stream(ctx)?;
97 let (logical, out_col_change) = self.rewrite_with_input(input, input_col_change);
98 Ok((logical.into(), out_col_change))
99 }
100
101 fn to_stream(&self, ctx: &mut ToStreamContext) -> Result<PlanRef> {
102 use super::stream::prelude::*;
103
104 let input = self.input().to_stream(ctx)?;
105 let input = RequiredDist::hash_shard(self.dedup_cols())
106 .enforce_if_not_satisfies(input, &Order::any())?;
107 if input.append_only() {
108 let mut logical_dedup = self.core.clone();
110 logical_dedup.input = input;
111 Ok(StreamDedup::new(logical_dedup).into())
112 } else {
113 let logical_top_n = generic::TopN::with_group(
115 input,
116 TopNLimit::new(1, false),
117 0,
118 Order::default(),
119 self.dedup_cols().to_vec(),
120 );
121 Ok(StreamGroupTopN::new(logical_top_n, None).into())
122 }
123 }
124}
125
126impl ToBatch for LogicalDedup {
127 fn to_batch(&self) -> Result<PlanRef> {
128 let input = self.input().to_batch()?;
129 let logical_top_n = generic::TopN::with_group(
130 input,
131 TopNLimit::new(1, false),
132 0,
133 Order::default(),
134 self.dedup_cols().to_vec(),
135 );
136 Ok(BatchGroupTopN::new(logical_top_n).into())
137 }
138}
139
140impl ExprRewritable for LogicalDedup {}
141
142impl ExprVisitable for LogicalDedup {}
143
144impl ColPrunable for LogicalDedup {
145 fn prune_col(&self, required_cols: &[usize], ctx: &mut ColumnPruningContext) -> PlanRef {
146 let input_required_bitset = FixedBitSet::from_iter(required_cols.iter().copied());
147 let dedup_required_bitset = {
148 let mut dedup_required_bitset = FixedBitSet::with_capacity(self.input().schema().len());
149 self.dedup_cols()
150 .iter()
151 .for_each(|idx| dedup_required_bitset.insert(*idx));
152 dedup_required_bitset
153 };
154 let input_required_cols = {
155 let mut tmp = input_required_bitset;
156 tmp.union_with(&dedup_required_bitset);
157 tmp.ones().collect_vec()
158 };
159 let mapping = ColIndexMapping::with_remaining_columns(
160 &input_required_cols,
161 self.input().schema().len(),
162 );
163
164 let new_input = self.input().prune_col(&input_required_cols, ctx);
165 let new_dedup_cols = self
166 .dedup_cols()
167 .iter()
168 .map(|&idx| mapping.map(idx))
169 .collect_vec();
170 let logical_dedup = Self::new(new_input, new_dedup_cols).into();
171
172 if input_required_cols == required_cols {
173 logical_dedup
174 } else {
175 let output_required_cols = required_cols
176 .iter()
177 .map(|&idx| mapping.map(idx))
178 .collect_vec();
179 let src_size = logical_dedup.schema().len();
180 LogicalProject::with_mapping(
181 logical_dedup,
182 ColIndexMapping::with_remaining_columns(&output_required_cols, src_size),
183 )
184 .into()
185 }
186 }
187}
188
189impl_distill_by_unit!(LogicalDedup, core, "LogicalDedup");