Skip to content

Commit

Permalink
planner: support predicate pushdown for Window (pingcap#11915)
Browse files Browse the repository at this point in the history
  • Loading branch information
francis0407 authored and sre-bot committed Aug 29, 2019
1 parent a81f8e3 commit 39f83c1
Show file tree
Hide file tree
Showing 4 changed files with 51 additions and 4 deletions.
11 changes: 11 additions & 0 deletions cmd/explaintest/r/explain_easy.result
Original file line number Diff line number Diff line change
Expand Up @@ -722,3 +722,14 @@ id count task operator info
IndexReader_12 10.00 root index:IndexScan_11
└─IndexScan_11 10.00 cop table:t, index:a, b, range:[1,1], keep order:true, stats:pseudo
drop table if exists t;
create table t(a int, b int);
explain select a, b from (select a, b, avg(b) over (partition by a)as avg_b from t) as tt where a > 10 and b < 10 and a > avg_b;
id count task operator info
Projection_8 2666.67 root test.t.a, test.t.b
└─Selection_9 2666.67 root gt(cast(test.t.a), avg_b), lt(test.t.b, 10)
└─Window_10 3333.33 root avg(cast(test.t.b)) over(partition by test.t.a)
└─Sort_14 3333.33 root test.t.a:asc
└─TableReader_13 3333.33 root data:Selection_12
└─Selection_12 3333.33 cop gt(test.t.a, 10)
└─TableScan_11 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo
drop table if exists t;
5 changes: 5 additions & 0 deletions cmd/explaintest/t/explain_easy.test
Original file line number Diff line number Diff line change
Expand Up @@ -173,3 +173,8 @@ create table t(a int, b int, index idx_ab(a, b));
explain select a, b from t where a in (1) order by b;
explain select a, b from t where a = 1 order by b;
drop table if exists t;

# https://github.com/pingcap/tidb/issues/11903
create table t(a int, b int);
explain select a, b from (select a, b, avg(b) over (partition by a)as avg_b from t) as tt where a > 10 and b < 10 and a > avg_b;
drop table if exists t;
13 changes: 12 additions & 1 deletion planner/core/logical_plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -200,8 +200,19 @@ func (s *testPlanSuite) TestPredicatePushDown(c *C) {
sql: "select * from t t1 join t t2 on t1.a = t2.a where t2.a = null",
best: "Dual->Projection",
},
{
sql: "select a, b from (select a, b, min(a) over(partition by b) as min_a from t)as tt where a < 10 and b > 10 and b = min_a",
best: "DataScan(t)->Projection->Projection->Window(min(test.t.a))->Sel([lt(test.tt.a, 10) eq(test.tt.b, 4_window_3)])->Projection->Projection",
},
{
sql: "select a, b from (select a, b, c, d, sum(a) over(partition by b, c) as sum_a from t)as tt where b + c > 10 and b in (1, 2) and sum_a > b",
best: "DataScan(t)->Projection->Projection->Window(sum(cast(test.t.a)))->Sel([gt(4_window_5, cast(test.tt.b))])->Projection->Projection",
},
}

s.Parser.EnableWindowFunc(true)
defer func() {
s.Parser.EnableWindowFunc(false)
}()
ctx := context.Background()
for ith, ca := range tests {
comment := Commentf("for %s", ca.sql)
Expand Down
26 changes: 23 additions & 3 deletions planner/core/rule_predicate_push_down.go
Original file line number Diff line number Diff line change
Expand Up @@ -532,11 +532,31 @@ func (p *LogicalJoin) outerJoinPropConst(predicates []expression.Expression) []e
return predicates
}

// getPartitionByCols extracts 'partition by' columns from the Window.
func (p *LogicalWindow) getPartitionByCols() []*expression.Column {
partitionCols := make([]*expression.Column, 0, len(p.PartitionBy))
for _, partitionItem := range p.PartitionBy {
partitionCols = append(partitionCols, partitionItem.Col)
}
return partitionCols
}

// PredicatePushDown implements LogicalPlan PredicatePushDown interface.
func (p *LogicalWindow) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) {
// Window function forbids any condition to push down.
p.baseLogicalPlan.PredicatePushDown(nil)
return predicates, p
canBePushed := make([]expression.Expression, 0, len(predicates))
canNotBePushed := make([]expression.Expression, 0, len(predicates))
partitionCols := expression.NewSchema(p.getPartitionByCols()...)
for _, cond := range predicates {
// We can push predicate beneath Window, only if all of the
// extractedCols are part of partitionBy columns.
if expression.ExprFromSchema(cond, partitionCols) {
canBePushed = append(canBePushed, cond)
} else {
canNotBePushed = append(canNotBePushed, cond)
}
}
p.baseLogicalPlan.PredicatePushDown(canBePushed)
return canNotBePushed, p
}

func (*ppdSolver) name() string {
Expand Down

0 comments on commit 39f83c1

Please sign in to comment.