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 TransformLimitToTopN #13288

Merged
merged 3 commits into from
Nov 9, 2019
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
7 changes: 7 additions & 0 deletions planner/cascades/testdata/transformation_rules_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -10,5 +10,12 @@
"select a, max(b) from t group by a having a > 1",
"select a, avg(b) from t group by a having a > 1 and max(b) > 10"
]
},
{
"name": "TestTopNRules",
"cases": [
"select b from t order by a limit 2",
"select a+b from t order by a limit 1 offset 2"
]
}
]
35 changes: 35 additions & 0 deletions planner/cascades/testdata/transformation_rules_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -110,5 +110,40 @@
]
}
]
},
{
"Name": "TestTopNRules",
"Cases": [
{
"SQL": "select b from t order by a limit 2",
"Result": [
"Group#0 Schema:[Column#15]",
" Projection_5 input:[Group#1], Column#13",
"Group#1 Schema:[Column#13,Column#14], UniqueKey:[Column#14]",
" TopN_8 input:[Group#2], Column#14:asc, offset:0, count:2",
"Group#2 Schema:[Column#13,Column#14], UniqueKey:[Column#14]",
" Projection_2 input:[Group#3], Column#2, Column#1",
"Group#3 Schema:[Column#1,Column#2], UniqueKey:[Column#1]",
" TableGather_7 input:[Group#4]",
"Group#4 Schema:[Column#1,Column#2], UniqueKey:[Column#1]",
" TableScan_6 table:t, pk col:Column#1"
]
},
{
"SQL": "select a+b from t order by a limit 1 offset 2",
"Result": [
"Group#0 Schema:[Column#15]",
" Projection_5 input:[Group#1], Column#13",
"Group#1 Schema:[Column#13,Column#14], UniqueKey:[Column#14]",
" TopN_8 input:[Group#2], Column#14:asc, offset:2, count:1",
"Group#2 Schema:[Column#13,Column#14], UniqueKey:[Column#14]",
" Projection_2 input:[Group#3], plus(Column#1, Column#2), Column#1",
"Group#3 Schema:[Column#1,Column#2], UniqueKey:[Column#1]",
" TableGather_7 input:[Group#4]",
"Group#4 Schema:[Column#1,Column#2], UniqueKey:[Column#1]",
" TableScan_6 table:t, pk col:Column#1"
]
}
]
}
]
40 changes: 40 additions & 0 deletions planner/cascades/transformation_rules.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ const (
rulePushSelDownProjection
rulePushSelDownAggregation
ruleEnumeratePaths
ruleTransformLimitToTopN
)

var transformationRuleList = []Transformation{
Expand All @@ -61,6 +62,7 @@ var transformationRuleList = []Transformation{
&PushSelDownProjection{},
&PushSelDownAggregation{},
&EnumeratePaths{},
&TransformLimitToTopN{},
}

var defaultTransformationMap = map[memo.Operand][]TransformationID{
Expand All @@ -74,6 +76,9 @@ var defaultTransformationMap = map[memo.Operand][]TransformationID{
memo.OperandDataSource: {
ruleEnumeratePaths,
},
memo.OperandLimit: {
ruleTransformLimitToTopN,
},
}

var patternMap []*memo.Pattern
Expand Down Expand Up @@ -411,3 +416,38 @@ func (r *PushSelDownAggregation) OnTransform(old *memo.ExprIter) (newExprs []*me
remainedGroupExpr.SetChildren(aggGroup)
return []*memo.GroupExpr{remainedGroupExpr}, true, false, nil
}

// TransformLimitToTopN transforms Limit+Sort to TopN.
type TransformLimitToTopN struct {
}

// GetPattern implements Transformation interface.
// The pattern of this rule is `Limit -> Sort`.
func (r *TransformLimitToTopN) GetPattern() *memo.Pattern {
return memo.BuildPattern(
memo.OperandLimit,
memo.EngineTiDBOnly,
memo.NewPattern(memo.OperandSort, memo.EngineTiDBOnly),
)
}

// Match implements Transformation interface.
func (r *TransformLimitToTopN) Match(expr *memo.ExprIter) bool {
return true
}

// OnTransform implements Transformation interface.
// This rule will transform `Limit -> Sort -> x` to `TopN -> x`.
func (r *TransformLimitToTopN) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) {
limit := old.GetExpr().ExprNode.(*plannercore.LogicalLimit)
sort := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalSort)
childGroup := old.Children[0].GetExpr().Children[0]
topN := plannercore.LogicalTopN{
ByItems: sort.ByItems,
Offset: limit.Offset,
Count: limit.Count,
}.Init(limit.SCtx(), limit.SelectBlockOffset())
topNExpr := memo.NewGroupExpr(topN)
topNExpr.SetChildren(childGroup)
return []*memo.GroupExpr{topNExpr}, true, false, nil
}
59 changes: 42 additions & 17 deletions planner/cascades/transformation_rules_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,30 @@ func (s *testTransformationRuleSuite) TearDownSuite(c *C) {
c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil)
}

func testGroupToString(input []string, output []struct {
SQL string
Result []string
}, s *testTransformationRuleSuite, c *C) {
for i, sql := range input {
stmt, err := s.ParseOneStmt(sql, "", "")
c.Assert(err, IsNil)
p, _, err := plannercore.BuildLogicalPlan(context.Background(), s.sctx, stmt, s.is)
c.Assert(err, IsNil)
logic, ok := p.(plannercore.LogicalPlan)
c.Assert(ok, IsTrue)
logic, err = s.optimizer.onPhasePreprocessing(s.sctx, logic)
c.Assert(err, IsNil)
group := convert2Group(logic)
err = s.optimizer.onPhaseExploration(s.sctx, group)
c.Assert(err, IsNil)
s.testData.OnRecord(func() {
output[i].SQL = sql
output[i].Result = ToString(group)
})
c.Assert(ToString(group), DeepEquals, output[i].Result)
}
}

func (s *testTransformationRuleSuite) TestPredicatePushDown(c *C) {
s.optimizer.ResetTransformationRules(map[memo.Operand][]TransformationID{
memo.OperandSelection: {
Expand All @@ -72,22 +96,23 @@ func (s *testTransformationRuleSuite) TestPredicatePushDown(c *C) {
Result []string
}
s.testData.GetTestCases(c, &input, &output)
for i, sql := range input {
stmt, err := s.ParseOneStmt(sql, "", "")
c.Assert(err, IsNil)
p, _, err := plannercore.BuildLogicalPlan(context.Background(), s.sctx, stmt, s.is)
c.Assert(err, IsNil)
logic, ok := p.(plannercore.LogicalPlan)
c.Assert(ok, IsTrue)
logic, err = s.optimizer.onPhasePreprocessing(s.sctx, logic)
c.Assert(err, IsNil)
group := convert2Group(logic)
err = s.optimizer.onPhaseExploration(s.sctx, group)
c.Assert(err, IsNil)
s.testData.OnRecord(func() {
output[i].SQL = sql
output[i].Result = ToString(group)
})
c.Assert(ToString(group), DeepEquals, output[i].Result)
testGroupToString(input, output, s, c)
}

func (s *testTransformationRuleSuite) TestTopNRules(c *C) {
s.optimizer.ResetTransformationRules(map[memo.Operand][]TransformationID{
memo.OperandLimit: {
ruleTransformLimitToTopN,
},
memo.OperandDataSource: {
ruleEnumeratePaths,
},
})
var input []string
var output []struct {
SQL string
Result []string
}
s.testData.GetTestCases(c, &input, &output)
testGroupToString(input, output, s, c)
}