Skip to content

Commit

Permalink
planner: fix wrong selection push down when having above agg (pingcap…
Browse files Browse the repository at this point in the history
  • Loading branch information
ti-srebot authored Sep 13, 2021
1 parent 647e246 commit fc92ef9
Show file tree
Hide file tree
Showing 6 changed files with 37 additions and 18 deletions.
9 changes: 9 additions & 0 deletions executor/aggregate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -892,6 +892,15 @@ func (s *testSuiteAgg) TestHaving(c *C) {
tk.MustQuery("select 1 from t group by c1 having sum(abs(c2 + c3)) = c1").Check(testkit.Rows("1"))
}

func (s *testSuiteAgg) TestIssue26496(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)

tk.MustExec("drop table if exists UK_NSPRE_19416")
tk.MustExec("CREATE TABLE `UK_NSPRE_19416` ( `COL1` binary(20) DEFAULT NULL, `COL2` varchar(20) DEFAULT NULL, `COL4` datetime DEFAULT NULL, `COL3` bigint(20) DEFAULT NULL, `COL5` float DEFAULT NULL, UNIQUE KEY `UK_COL1` (`COL1`)) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")
tk.MustExec("insert into `UK_NSPRE_19416`(col1) values (0xc5b428e2ebc1b78f0b183899a8df55c88a333f86), (0x004dad637b37cc4a9742484ab93f97ede2ab8bd5), (0x550c4a4390ba14fd6d382dd29063e10210c99381)")
tk.MustQuery("select t1.col1, count(t2.col1) from UK_NSPRE_19416 as t1 left join UK_NSPRE_19416 as t2 on t1.col1 = t2.col1 where t1.col1 in (0x550C4A4390BA14FD6D382DD29063E10210C99381, 0x004DAD637B37CC4A9742484AB93F97EDE2AB8BD5, 0xC5B428E2EBC1B78F0B183899A8DF55C88A333F86) group by t1.col1, t2.col1 having t1.col1 in (0x9B4B48FEBA9225BACF8F9ADEAEE810AEC26DC7A2, 0x25A6C4FAD832F8E0267AAA504CFAE767565C8B84, 0xE26E5B0080EC5A8156DACE67D13B239500E540E6)").Check(nil)
}

func (s *testSuiteAgg) TestAggEliminator(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)

Expand Down
6 changes: 3 additions & 3 deletions planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -916,17 +916,17 @@ func (b *PlanBuilder) coalesceCommonColumns(p *LogicalJoin, leftPlan, rightPlan
return nil
}

func (b *PlanBuilder) buildSelection(ctx context.Context, p LogicalPlan, where ast.ExprNode, AggMapper map[*ast.AggregateFuncExpr]int) (LogicalPlan, error) {
func (b *PlanBuilder) buildSelection(ctx context.Context, p LogicalPlan, where ast.ExprNode, aggMapper map[*ast.AggregateFuncExpr]int) (LogicalPlan, error) {
b.optFlag |= flagPredicatePushDown
if b.curClause != havingClause {
b.curClause = whereClause
}

conditions := splitWhere(where)
expressions := make([]expression.Expression, 0, len(conditions))
selection := LogicalSelection{}.Init(b.ctx, b.getSelectOffset())
selection := LogicalSelection{buildByHaving: aggMapper != nil}.Init(b.ctx, b.getSelectOffset())
for _, cond := range conditions {
expr, np, err := b.rewrite(ctx, cond, p, AggMapper, false)
expr, np, err := b.rewrite(ctx, cond, p, aggMapper, false)
if err != nil {
return nil, err
}
Expand Down
3 changes: 3 additions & 0 deletions planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -395,6 +395,9 @@ type LogicalSelection struct {
// but after we converted to CNF(Conjunctive normal form), it can be
// split into a list of AND conditions.
Conditions []expression.Expression

// having selection can't be pushed down, because it must above the aggregation.
buildByHaving bool
}

// ExtractCorrelatedCols implements LogicalPlan interface.
Expand Down
13 changes: 10 additions & 3 deletions planner/core/rule_predicate_push_down.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,9 +73,16 @@ func splitSetGetVarFunc(filters []expression.Expression) ([]expression.Expressio

// PredicatePushDown implements LogicalPlan PredicatePushDown interface.
func (p *LogicalSelection) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) {
canBePushDown, canNotBePushDown := splitSetGetVarFunc(p.Conditions)
retConditions, child := p.children[0].PredicatePushDown(append(canBePushDown, predicates...))
retConditions = append(retConditions, canNotBePushDown...)
var child LogicalPlan
var retConditions []expression.Expression
if p.buildByHaving {
retConditions, child = p.children[0].PredicatePushDown(predicates)
retConditions = append(retConditions, p.Conditions...)
} else {
canBePushDown, canNotBePushDown := splitSetGetVarFunc(p.Conditions)
retConditions, child = p.children[0].PredicatePushDown(append(canBePushDown, predicates...))
retConditions = append(retConditions, canNotBePushDown...)
}
if len(retConditions) > 0 {
p.Conditions = expression.PropagateConstant(p.ctx, retConditions)
// Return table dual when filter is constant false or null.
Expand Down
12 changes: 6 additions & 6 deletions planner/core/testdata/ordered_result_mode_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -396,12 +396,12 @@
},
{
"Plan": [
"Sort_9 6400.00 root Column#5:asc, Column#6:asc, Column#7:asc",
"└─Selection_11 6400.00 root lt(Column#6, 20)",
" └─HashAgg_16 8000.00 root group by:test.t1.d, funcs:min(Column#11)->Column#5, funcs:max(Column#12)->Column#6, funcs:sum(Column#13)->Column#7",
" └─TableReader_17 8000.00 root data:HashAgg_12",
" └─HashAgg_12 8000.00 cop[tikv] group by:test.t1.d, funcs:min(test.t1.a)->Column#11, funcs:max(test.t1.b)->Column#12, funcs:sum(test.t1.c)->Column#13",
" └─TableFullScan_15 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo"
"Selection_8 6400.00 root lt(Column#6, 20)",
"└─Sort_9 8000.00 root Column#5:asc, Column#6:asc, Column#7:asc",
" └─HashAgg_15 8000.00 root group by:test.t1.d, funcs:min(Column#11)->Column#5, funcs:max(Column#12)->Column#6, funcs:sum(Column#13)->Column#7",
" └─TableReader_16 8000.00 root data:HashAgg_11",
" └─HashAgg_11 8000.00 cop[tikv] group by:test.t1.d, funcs:min(test.t1.a)->Column#11, funcs:max(test.t1.b)->Column#12, funcs:sum(test.t1.c)->Column#13",
" └─TableFullScan_14 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo"
]
},
{
Expand Down
12 changes: 6 additions & 6 deletions planner/core/testdata/plan_suite_unexported_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
"Name": "TestEagerAggregation",
"Cases": [
"DataScan(t)->Aggr(sum(test.t.a),sum(plus(test.t.a, 1)),count(test.t.a))->Projection",
"DataScan(t)->Aggr(sum(plus(test.t.a, test.t.b)),sum(plus(test.t.a, test.t.c)),count(test.t.a))->Sel([gt(Column#13, 0)])->Projection->Sort->Projection",
"DataScan(t)->Aggr(sum(plus(test.t.a, test.t.b)),sum(plus(test.t.a, test.t.c)),count(test.t.a))->Projection->Sel([gt(Column#13, 0)])->Sort->Projection",
"Join{DataScan(a)->Aggr(sum(test.t.a),firstrow(test.t.c))->DataScan(b)}(test.t.c,test.t.c)->Aggr(sum(Column#26))->Projection",
"Join{DataScan(a)->DataScan(b)->Aggr(sum(test.t.a),firstrow(test.t.c))}(test.t.c,test.t.c)->Aggr(sum(Column#26))->Projection",
"Join{DataScan(a)->DataScan(b)->Aggr(sum(test.t.a),firstrow(test.t.c))}(test.t.c,test.t.c)->Aggr(sum(Column#26),firstrow(test.t.a))->Projection",
Expand Down Expand Up @@ -89,7 +89,7 @@
"DataScan(t)->Aggr(sum(test.t.b),firstrow(test.t.a))->Sel([gt(cast(test.t.a, decimal(20,0) BINARY), Column#13)])->Projection->Projection",
"DataScan(t)->Aggr(sum(test.t.b),firstrow(test.t.a))->Sel([gt(test.t.a, 1)])->Projection->Projection",
"Dual->Sel([gt(test.t.a, 1)])->Projection",
"DataScan(t)->Aggr(count(test.t.a),firstrow(test.t.a))->Sel([lt(Column#13, 1)])->Projection",
"DataScan(t)->Aggr(count(test.t.a),firstrow(test.t.a))->Projection->Sel([lt(Column#13, 1)])",
"Join{DataScan(t1)->DataScan(t2)}(test.t.a,test.t.a)->Projection",
"Dual->Projection",
"DataScan(t)->Projection->Projection->Window(min(test.t.a)->Column#14)->Sel([lt(test.t.a, 10) eq(test.t.b, Column#14)])->Projection->Projection",
Expand Down Expand Up @@ -194,7 +194,7 @@
"TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over())->Sort->Projection",
"TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over(partition by test.t.a))->Sort->Projection",
"TableReader(Table(t)->StreamAgg)->StreamAgg->Window(sum(Column#13)->Column#15 over())->Sort->Projection",
"Apply{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.f)[[NULL,+inf]]->Sel([gt(test.t.a, test.t.a)]))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#38 over())->MaxOneRow->Sel([Column#38])}->Projection",
"Apply{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.f)[[NULL,+inf]]->Sel([gt(test.t.a, test.t.a)]))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#38 over())->MaxOneRow}->Sel([Column#38])->Projection",
"[planner:3594]You cannot use the alias 'w' of an expression containing a window function in this context.'",
"[planner:1247]Reference 'sum_a' not supported (reference to window function)",
"[planner:3579]Window name 'w2' is not defined.",
Expand Down Expand Up @@ -267,7 +267,7 @@
"TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over())->Sort->Projection",
"TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over(partition by test.t.a))->Sort->Projection",
"TableReader(Table(t)->StreamAgg)->StreamAgg->Window(sum(Column#13)->Column#15 over())->Sort->Projection",
"Apply{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.f)[[NULL,+inf]]->Sel([gt(test.t.a, test.t.a)]))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#38 over())->MaxOneRow->Sel([Column#38])}->Projection",
"Apply{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.f)[[NULL,+inf]]->Sel([gt(test.t.a, test.t.a)]))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#38 over())->MaxOneRow}->Sel([Column#38])->Projection",
"[planner:3594]You cannot use the alias 'w' of an expression containing a window function in this context.'",
"[planner:1247]Reference 'sum_a' not supported (reference to window function)",
"[planner:3579]Window name 'w2' is not defined.",
Expand Down Expand Up @@ -493,12 +493,12 @@
"test.t.f"
]
],
"5": [
"4": [
[
"test.t.f"
]
],
"6": [
"5": [
[
"test.t.f"
]
Expand Down

0 comments on commit fc92ef9

Please sign in to comment.