Skip to content

Commit 9b89bb0

Browse files
authored
fix(optimizer): projectSet && overAgg should call input's predicate push down && prune col (risingwavelabs#8588)
1 parent 14bfc62 commit 9b89bb0

File tree

4 files changed

+23
-12
lines changed

4 files changed

+23
-12
lines changed

src/frontend/planner_test/tests/testdata/over_window_function.yaml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -294,7 +294,7 @@
294294
└─StreamHashAgg { group_key: [$expr1, $expr2, bid.supplier_id], aggs: [sum(bid.price), count] }
295295
└─StreamExchange { dist: HashShard($expr1, $expr2, bid.supplier_id) }
296296
└─StreamProject { exprs: [TumbleStart(bid.bidtime, '00:10:00':Interval) as $expr1, (TumbleStart(bid.bidtime, '00:10:00':Interval) + '00:10:00':Interval) as $expr2, bid.supplier_id, bid.price, bid._row_id] }
297-
└─StreamTableScan { table: bid, columns: [bid.bidtime, bid.price, bid.item, bid.supplier_id, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) }
297+
└─StreamTableScan { table: bid, columns: [bid.bidtime, bid.price, bid.supplier_id, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) }
298298
- before:
299299
- create_bid
300300
sql: |

src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -129,7 +129,7 @@
129129
└─LogicalAgg { group_key: [t.v1, t.v2, t.v3], aggs: [count, count(1:Int32)] }
130130
└─LogicalProject { exprs: [t.v1, t.v2, t.v3, 1:Int32] }
131131
└─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3], predicate: (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) AND (t.v2 > t.v3) }
132-
- name: filter project set transpose
132+
- name: filter project set transpose TODO(https://github.com/risingwavelabs/risingwave/issues/8591)
133133
sql: |
134134
create table t(v1 int, v2 int, v3 int, arr int[]);
135135
with cte as (select v1, v2, v3, unnest(arr) as arr_unnested from t) select * from cte where v1=10 AND v2=20 AND v3=30 AND arr_unnested=30;

src/frontend/src/optimizer/plan_node/logical_over_agg.rs

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -252,9 +252,14 @@ impl fmt::Display for LogicalOverAgg {
252252
}
253253

254254
impl ColPrunable for LogicalOverAgg {
255-
fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef {
255+
fn prune_col(&self, required_cols: &[usize], ctx: &mut ColumnPruningContext) -> PlanRef {
256256
let mapping = ColIndexMapping::with_remaining_columns(required_cols, self.schema().len());
257-
LogicalProject::with_mapping(self.clone().into(), mapping).into()
257+
let new_input = {
258+
let input = self.input();
259+
let required = (0..input.schema().len()).collect_vec();
260+
input.prune_col(&required, ctx)
261+
};
262+
LogicalProject::with_mapping(self.clone_with_input(new_input).into(), mapping).into()
258263
}
259264
}
260265

src/frontend/src/optimizer/plan_node/logical_project_set.rs

Lines changed: 14 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -14,11 +14,12 @@
1414

1515
use std::fmt;
1616

17+
use itertools::Itertools;
1718
use risingwave_common::error::Result;
1819

1920
use super::{
20-
generic, BatchProjectSet, ColPrunable, ExprRewritable, LogicalFilter, LogicalProject, PlanBase,
21-
PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, ToBatch, ToStream,
21+
gen_filter_and_pushdown, generic, BatchProjectSet, ColPrunable, ExprRewritable, LogicalProject,
22+
PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, ToBatch, ToStream,
2223
};
2324
use crate::expr::{Expr, ExprImpl, ExprRewriter, FunctionCall, InputRef, TableFunction};
2425
use crate::optimizer::plan_node::{
@@ -237,10 +238,15 @@ impl fmt::Display for LogicalProjectSet {
237238
}
238239

239240
impl ColPrunable for LogicalProjectSet {
240-
fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef {
241-
// TODO: column pruning for ProjectSet
241+
fn prune_col(&self, required_cols: &[usize], ctx: &mut ColumnPruningContext) -> PlanRef {
242+
// TODO: column pruning for ProjectSet https://github.com/risingwavelabs/risingwave/issues/8593
242243
let mapping = ColIndexMapping::with_remaining_columns(required_cols, self.schema().len());
243-
LogicalProject::with_mapping(self.clone().into(), mapping).into()
244+
let new_input = {
245+
let input = self.input();
246+
let required = (0..input.schema().len()).collect_vec();
247+
input.prune_col(&required, ctx)
248+
};
249+
LogicalProject::with_mapping(self.clone_with_input(new_input).into(), mapping).into()
244250
}
245251
}
246252

@@ -264,10 +270,10 @@ impl PredicatePushdown for LogicalProjectSet {
264270
fn predicate_pushdown(
265271
&self,
266272
predicate: Condition,
267-
_ctx: &mut PredicatePushdownContext,
273+
ctx: &mut PredicatePushdownContext,
268274
) -> PlanRef {
269-
// TODO: predicate pushdown for ProjectSet
270-
LogicalFilter::create(self.clone().into(), predicate)
275+
// TODO: predicate pushdown https://github.com/risingwavelabs/risingwave/issues/8591
276+
gen_filter_and_pushdown(self, predicate, Condition::true_cond(), ctx)
271277
}
272278
}
273279

0 commit comments

Comments
 (0)