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

planner/cascades: add transformation rule EliminateSingleMaxMin #14274

Merged
merged 18 commits into from
Feb 5, 2020
Merged
Show file tree
Hide file tree
Changes from 13 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
1 change: 1 addition & 0 deletions planner/cascades/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,6 +152,7 @@ func (s *testIntegrationSuite) TestAggregation(c *C) {
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int primary key, b int)")
tk.MustExec("insert into t values (1, 11), (4, 44), (2, 22), (3, 33)")
tk.MustExec("set @@session.sql_mode=''")
tk.MustExec("set session tidb_enable_cascades_planner = 1")
var input []string
var output []struct {
Expand Down
6 changes: 5 additions & 1 deletion planner/cascades/testdata/integration_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,11 @@
"select b, sum(a) from t group by b order by b",
"select b, avg(a) from t group by b having sum(a) > 1 order by b",
"select max(a+b) from t",
"select sum(a) from t group by a, a+b order by a"
"select sum(a) from t group by a, a+b order by a",
"select max(a) from t",
"select max(a), b from t",
"select min(a) from t",
"select a, min(b) from t"
]
},
{
Expand Down
60 changes: 56 additions & 4 deletions planner/cascades/testdata/integration_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -212,10 +212,10 @@
{
"SQL": "select max(a+b) from t",
"Plan": [
"HashAgg_12 1.00 root funcs:max(Column#4)->Column#3",
"└─TableReader_13 1.00 root data:HashAgg_14",
" └─HashAgg_14 1.00 cop[tikv] funcs:max(plus(test.t.a, test.t.b))->Column#4",
" └─TableScan_10 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo"
"HashAgg_16 1.00 root funcs:max(Column#4)->Column#3",
"└─TableReader_17 1.00 root data:HashAgg_18",
" └─HashAgg_18 1.00 cop[tikv] funcs:max(plus(test.t.a, test.t.b))->Column#4",
" └─TableScan_14 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo"
],
"Result": [
"48"
Expand All @@ -237,6 +237,58 @@
"3",
"4"
]
},
{
"SQL": "select max(a) from t",
"Plan": [
"HashAgg_16 1.00 root funcs:max(test.t.a)->Column#3",
"└─Limit_18 1.00 root offset:0, count:1",
" └─TableReader_23 1.00 root data:Limit_24",
" └─Limit_24 1.00 cop[tikv] offset:0, count:1",
" └─TableScan_22 1.00 cop[tikv] table:t, range:[-inf,+inf], keep order:true, desc, stats:pseudo"
],
"Result": [
"4"
]
},
{
"SQL": "select max(a), b from t",
"Plan": [
"HashAgg_16 1.00 root funcs:max(test.t.a)->Column#3, funcs:firstrow(test.t.b)->test.t.b",
"└─Limit_18 1.00 root offset:0, count:1",
" └─TableReader_23 1.00 root data:Limit_24",
" └─Limit_24 1.00 cop[tikv] offset:0, count:1",
" └─TableScan_22 1.00 cop[tikv] table:t, range:[-inf,+inf], keep order:true, desc, stats:pseudo"
],
"Result": [
"4 44"
]
},
{
"SQL": "select min(a) from t",
"Plan": [
"HashAgg_16 1.00 root funcs:min(test.t.a)->Column#3",
"└─Limit_18 1.00 root offset:0, count:1",
" └─TableReader_23 1.00 root data:Limit_24",
" └─Limit_24 1.00 cop[tikv] offset:0, count:1",
" └─TableScan_22 1.00 cop[tikv] table:t, range:[-inf,+inf], keep order:true, stats:pseudo"
],
"Result": [
"1"
]
},
{
"SQL": "select a, min(b) from t",
"Plan": [
"Projection_12 1.00 root test.t.a, Column#3",
"└─HashAgg_28 1.00 root funcs:min(Column#4)->Column#3, funcs:firstrow(Column#5)->test.t.a",
" └─TableReader_29 1.00 root data:HashAgg_30",
" └─HashAgg_30 1.00 cop[tikv] funcs:min(test.t.b)->Column#4, funcs:firstrow(test.t.a)->Column#5",
" └─TableScan_15 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo"
],
"Result": [
"1 11"
]
}
]
},
Expand Down
9 changes: 9 additions & 0 deletions planner/cascades/testdata/transformation_rules_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,15 @@
"select a+c from (select floor(a) as a, b, c from t) as t2"
]
},
{
"name": "TestEliminateMaxMin",
"cases": [
"select max(a) from t;",
"select max(a), b from t;",
"select min(a) from t",
"select a, min(b) from t"
]
},
{
"name": "TestMergeAggregationProjection",
"cases": [
Expand Down
61 changes: 61 additions & 0 deletions planner/cascades/testdata/transformation_rules_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -1088,6 +1088,67 @@
}
]
},
{
"Name": "TestEliminateMaxMin",
"Cases": [
{
"SQL": "select max(a) from t;",
"Result": [
"Group#0 Schema:[Column#13]",
" Projection_3 input:[Group#1], Column#13",
"Group#1 Schema:[Column#13]",
" Aggregation_2 input:[Group#2], funcs:max(test.t.a)",
" Aggregation_2 input:[Group#3], funcs:max(test.t.a)",
"Group#2 Schema:[test.t.a]",
" TableScan_1 table:t",
"Group#3 Schema:[test.t.a]",
" TopN_4 input:[Group#2], test.t.a:desc, offset:0, count:1"
]
},
{
"SQL": "select max(a), b from t;",
"Result": [
"Group#0 Schema:[Column#13,test.t.b]",
" Projection_3 input:[Group#1], Column#13, test.t.b",
"Group#1 Schema:[Column#13,test.t.b]",
" Aggregation_2 input:[Group#2], funcs:max(test.t.a), firstrow(test.t.b)",
" Aggregation_2 input:[Group#3], funcs:max(test.t.a), firstrow(test.t.b)",
"Group#2 Schema:[test.t.a,test.t.b]",
" TableScan_1 table:t",
"Group#3 Schema:[test.t.a,test.t.b]",
" TopN_4 input:[Group#2], test.t.a:desc, offset:0, count:1"
]
},
{
"SQL": "select min(a) from t",
"Result": [
"Group#0 Schema:[Column#13]",
" Projection_3 input:[Group#1], Column#13",
"Group#1 Schema:[Column#13]",
" Aggregation_2 input:[Group#2], funcs:min(test.t.a)",
" Aggregation_2 input:[Group#3], funcs:min(test.t.a)",
"Group#2 Schema:[test.t.a]",
" TableScan_1 table:t",
"Group#3 Schema:[test.t.a]",
" TopN_4 input:[Group#2], test.t.a:asc, offset:0, count:1"
]
},
{
"SQL": "select a, min(b) from t",
"Result": [
"Group#0 Schema:[test.t.a,Column#13]",
" Projection_3 input:[Group#1], test.t.a, Column#13",
"Group#1 Schema:[Column#13,test.t.a]",
" Aggregation_2 input:[Group#2], funcs:min(test.t.b), firstrow(test.t.a)",
" Aggregation_2 input:[Group#3], funcs:min(test.t.b), firstrow(test.t.a)",
"Group#2 Schema:[test.t.a,test.t.b]",
" TableScan_1 table:t",
"Group#3 Schema:[test.t.a,test.t.b]",
" TopN_4 input:[Group#2], test.t.b:asc, offset:0, count:1"
]
}
]
},
{
"Name": "TestMergeAggregationProjection",
"Cases": [
Expand Down
118 changes: 118 additions & 0 deletions planner/cascades/transformation_rules.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,13 @@ package cascades
import (
"math"

"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/expression/aggregation"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/planner/memo"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/ranger"
)

Expand Down Expand Up @@ -63,6 +66,7 @@ var defaultTransformationMap = map[memo.Operand][]Transformation{
memo.OperandAggregation: {
NewRulePushAggDownGather(),
NewRuleMergeAggregationProjection(),
NewRuleEliminateSingleMaxMin(),
},
memo.OperandLimit: {
NewRuleTransformLimitToTopN(),
Expand Down Expand Up @@ -1342,6 +1346,120 @@ func (r *MergeAggregationProjection) OnTransform(old *memo.ExprIter) (newExprs [
return []*memo.GroupExpr{newAggExpr}, true, false, nil
}

// EliminateSingleMaxMin tries to convert a single max/min to Limit+Sort operators.
type EliminateSingleMaxMin struct {
baseRule
}

// NewRuleEliminateSingleMaxMin creates a new Transformation EliminateSingleMaxMin.
// The pattern of this rule is `max/min->X`.
func NewRuleEliminateSingleMaxMin() Transformation {
rule := &EliminateSingleMaxMin{}
rule.pattern = memo.BuildPattern(
memo.OperandAggregation,
memo.EngineTiDBOnly,
memo.NewPattern(memo.OperandAny, memo.EngineTiDBOnly),
)
return rule
}

// Match implements Transformation interface.
func (r *EliminateSingleMaxMin) Match(expr *memo.ExprIter) bool {
// Use appliedRuleSet in GroupExpr to avoid re-apply rules.
if expr.GetExpr().HasAppliedRule(r) {
return false
}

agg := expr.GetExpr().ExprNode.(*plannercore.LogicalAggregation)
// EliminateSingleMaxMin only works on the complete mode.
if !agg.IsCompleteModeAgg() {
return false
}
if len(agg.GroupByItems) != 0 {
return false
}

// Only one max() or min() in the AggFuncs slice, and
// the other aggregate functions in the AggFuncs slice should be FirstRow().
Copy link
Contributor

@alivxxx alivxxx Jan 19, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we really elimate max/min in this case? After transformation, there is a selection between source and agg, so the first row becomes first row that the max/min arguments is not null, is it expected?

Copy link
Contributor Author

@Reminiscent Reminiscent Jan 20, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's unexpected. In mysql5.7:

mysql> select * from t;
+------+------+
| a    | b    |
+------+------+
| NULL |    1 |
|    1 |    2 |
+------+------+
2 rows in set (0.00 sec)

mysql> select max(a), b from t;
+--------+------+
| max(a) | b    |
+--------+------+
|      1 |    1 |
+--------+------+
1 row in set (0.00 sec)

I'll fix it. Thanks for your review.

Copy link
Contributor Author

@Reminiscent Reminiscent Jan 20, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems it's not easy to include the first_row() aggFun in this transformation rule. Or we can add a check in the match function. We can't use this rule when the two conditions that first_row() and the column for max/min aggFunc can be NULL, exist at the same time. But I think it has too many restrictions leading to increased complexity. What's your opinion? @francis0407 @lamxTyler @zz-jason

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@lamxTyler good catch !
I think we can revert this new feature. It’s hard to do that in the current framework. Just transform max/min like what we did before.
cc @zz-jason

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we can rewrite the Match() condition to this:

  1. only one max or min, we can apply this transformation
  2. only one max or min, and all the max or min arguments are not null, we can apply this transformation as well.

BTW, how do calcite handle this problem?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Another problem is that currently we cannot check if the arguments are not null. mysql.HashNotNullFlag does not work here, since operators like OuterJoin may generate null values on not null columns.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can add a TODO here, waiting for the not null prop is manitained.

maxMinFlag := false
Reminiscent marked this conversation as resolved.
Show resolved Hide resolved
for _, aggFunc := range agg.AggFuncs {
if aggFunc.Name == ast.AggFuncMax || aggFunc.Name == ast.AggFuncMin {
if maxMinFlag {
return false
}
maxMinFlag = true
} else {
if aggFunc.Name != ast.AggFuncFirstRow {
return false
}
}
}
return true
}

// OnTransform implements Transformation interface.
// It will transform `max/min->X` to `max/min->top1->sel->X`.
func (r *EliminateSingleMaxMin) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) {
agg := old.GetExpr().ExprNode.(*plannercore.LogicalAggregation)
childGroup := old.GetExpr().Children[0]
ctx := agg.SCtx()

var maxMinPos int
for i, aggFunc := range agg.AggFuncs {
if aggFunc.Name == ast.AggFuncMax || aggFunc.Name == ast.AggFuncMin {
maxMinPos = i
break
}
}
f := agg.AggFuncs[maxMinPos]

// If there's no column in f.GetArgs()[0], we still need limit and read data from real table because the result should be NULL if the input is empty.
if len(expression.ExtractColumns(f.Args[maxMinPos])) > 0 {
// If it can be NULL, we need to filter NULL out first.
if !mysql.HasNotNullFlag(f.Args[maxMinPos].GetType().Flag) {
sel := plannercore.LogicalSelection{}.Init(ctx, agg.SelectBlockOffset())
isNullFunc := expression.NewFunctionInternal(ctx, ast.IsNull, types.NewFieldType(mysql.TypeTiny), f.Args[maxMinPos])
notNullFunc := expression.NewFunctionInternal(ctx, ast.UnaryNot, types.NewFieldType(mysql.TypeTiny), isNullFunc)
sel.Conditions = []expression.Expression{notNullFunc}
selExpr := memo.NewGroupExpr(sel)
selExpr.SetChildren(childGroup)
selGroup := memo.NewGroupWithSchema(selExpr, childGroup.Prop.Schema)
childGroup = selGroup
}

// Add top(1) operators.
// For max function, the sort order should be desc.
desc := f.Name == ast.AggFuncMax
var byItems []*plannercore.ByItems
byItems = append(byItems, &plannercore.ByItems{
Expr: f.Args[maxMinPos],
Desc: desc,
})
top1 := plannercore.LogicalTopN{
ByItems: byItems,
Count: 1,
}.Init(ctx, agg.SelectBlockOffset())
top1Expr := memo.NewGroupExpr(top1)
top1Expr.SetChildren(childGroup)
top1Group := memo.NewGroupWithSchema(top1Expr, childGroup.Prop.Schema)
childGroup = top1Group
} else {
li := plannercore.LogicalLimit{Count: 1}.Init(ctx, agg.SelectBlockOffset())
liExpr := memo.NewGroupExpr(li)
liExpr.SetChildren(childGroup)
liGroup := memo.NewGroupWithSchema(liExpr, childGroup.Prop.Schema)
childGroup = liGroup
}

newAgg := agg
newAggExpr := memo.NewGroupExpr(newAgg)
// If no data in the child, we need to return NULL instead of empty. This cannot be done by sort and limit themselves.
// Since now there would be at most one row returned, the remained agg operator is not expensive anymore.
newAggExpr.SetChildren(childGroup)
newAggExpr.AddAppliedRule(r)
return []*memo.GroupExpr{newAggExpr}, false, false, nil
}

// MergeAdjacentSelection merge adjacent selection.
type MergeAdjacentSelection struct {
baseRule
Expand Down
18 changes: 18 additions & 0 deletions planner/cascades/transformation_rules_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -191,6 +191,24 @@ func (s *testTransformationRuleSuite) TestProjectionElimination(c *C) {
testGroupToString(input, output, s, c)
}

func (s *testTransformationRuleSuite) TestEliminateMaxMin(c *C) {
s.optimizer.ResetTransformationRules(map[memo.Operand][]Transformation{
memo.OperandAggregation: {
NewRuleEliminateSingleMaxMin(),
},
})
defer func() {
s.optimizer.ResetTransformationRules(defaultTransformationMap)
}()
var input []string
var output []struct {
SQL string
Result []string
}
s.testData.GetTestCases(c, &input, &output)
testGroupToString(input, output, s, c)
}

func (s *testTransformationRuleSuite) TestMergeAggregationProjection(c *C) {
s.optimizer.ResetTransformationRules(map[memo.Operand][]Transformation{
memo.OperandAggregation: {
Expand Down
6 changes: 6 additions & 0 deletions planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -291,6 +291,12 @@ func (la *LogicalAggregation) IsPartialModeAgg() bool {
return la.AggFuncs[0].Mode == aggregation.Partial1Mode
}

// IsCompleteModeAgg returns if all of the AggFuncs are CompleteMode.
func (la *LogicalAggregation) IsCompleteModeAgg() bool {
// Since all of the AggFunc share the same AggMode, we only need to check the first one.
return la.AggFuncs[0].Mode == aggregation.CompleteMode
}

// GetGroupByCols returns the groupByCols. If the groupByCols haven't be collected,
// this method would collect them at first. If the GroupByItems have been changed,
// we should explicitly collect GroupByColumns before this method.
Expand Down