Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

fix(optimizer): projectSet && overAgg should call input's predicate push down && prune col #8588

Merged
merged 3 commits into from
Mar 16, 2023
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -294,7 +294,7 @@
└─StreamHashAgg { group_key: [$expr1, $expr2, bid.supplier_id], aggs: [sum(bid.price), count] }
└─StreamExchange { dist: HashShard($expr1, $expr2, bid.supplier_id) }
└─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] }
└─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) }
└─StreamTableScan { table: bid, columns: [bid.bidtime, bid.price, bid.supplier_id, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) }
- before:
- create_bid
sql: |
Expand Down
9 changes: 7 additions & 2 deletions src/frontend/src/optimizer/plan_node/logical_over_agg.rs
Original file line number Diff line number Diff line change
Expand Up @@ -252,9 +252,14 @@ impl fmt::Display for LogicalOverAgg {
}

impl ColPrunable for LogicalOverAgg {
fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef {
fn prune_col(&self, required_cols: &[usize], ctx: &mut ColumnPruningContext) -> PlanRef {
let mapping = ColIndexMapping::with_remaining_columns(required_cols, self.schema().len());
LogicalProject::with_mapping(self.clone().into(), mapping).into()
let new_input = {
let input = self.input();
let required = (0..input.schema().len()).collect_vec();
input.prune_col(&required, ctx)
};
LogicalProject::with_mapping(self.clone_with_input(new_input).into(), mapping).into()
}
}

Expand Down
20 changes: 12 additions & 8 deletions src/frontend/src/optimizer/plan_node/logical_project_set.rs
Original file line number Diff line number Diff line change
Expand Up @@ -14,11 +14,12 @@

use std::fmt;

use itertools::Itertools;
use risingwave_common::error::Result;

use super::{
generic, BatchProjectSet, ColPrunable, ExprRewritable, LogicalFilter, LogicalProject, PlanBase,
PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, ToBatch, ToStream,
gen_filter_and_pushdown, generic, BatchProjectSet, ColPrunable, ExprRewritable, LogicalProject,
PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, ToBatch, ToStream,
};
use crate::expr::{Expr, ExprImpl, ExprRewriter, FunctionCall, InputRef, TableFunction};
use crate::optimizer::plan_node::{
Expand Down Expand Up @@ -237,10 +238,14 @@ impl fmt::Display for LogicalProjectSet {
}

impl ColPrunable for LogicalProjectSet {
fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef {
// TODO: column pruning for ProjectSet
fn prune_col(&self, required_cols: &[usize], ctx: &mut ColumnPruningContext) -> PlanRef {
let mapping = ColIndexMapping::with_remaining_columns(required_cols, self.schema().len());
LogicalProject::with_mapping(self.clone().into(), mapping).into()
let new_input = {
let input = self.input();
let required = (0..input.schema().len()).collect_vec();
input.prune_col(&required, ctx)
};
LogicalProject::with_mapping(self.clone_with_input(new_input).into(), mapping).into()
}
}

Expand All @@ -264,10 +269,9 @@ impl PredicatePushdown for LogicalProjectSet {
fn predicate_pushdown(
&self,
predicate: Condition,
_ctx: &mut PredicatePushdownContext,
ctx: &mut PredicatePushdownContext,
) -> PlanRef {
// TODO: predicate pushdown for ProjectSet
LogicalFilter::create(self.clone().into(), predicate)
gen_filter_and_pushdown(self, predicate, Condition::true_cond(), ctx)
}
}

Expand Down