Skip to content

Commit

Permalink
planner/cascades: add transformation rule PushTopNDownUnionAll (#14214)
Browse files Browse the repository at this point in the history
  • Loading branch information
Reminiscent authored and sre-bot committed Dec 25, 2019
1 parent aa2f716 commit 980f72d
Show file tree
Hide file tree
Showing 4 changed files with 272 additions and 1 deletion.
6 changes: 5 additions & 1 deletion planner/cascades/testdata/transformation_rules_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,11 @@
"select c from t order by t.a + t.b limit 1",
"select a, b, c from t t1 where t1.a in (select t2.a as a from t t2 where t2.b > t1.b order by t1.b limit 1)",
"select a, b, c from t t1 where t1.a in (select a from (select t2.a as a, t1.b as b from t t2 where t2.b > t1.b) x order by b limit 1)",
"select a, b from (select @i as a, @i := @i+1 as b from t) t order by a desc limit 1"
"select a, b from (select @i as a, @i := @i+1 as b from t) t order by a desc limit 1",
"(select a from t) union all (select b from t) order by a limit 2;",
"(select a from t) union all (select sum(a) from t where a > 2 group by b) order by a limit 2;",
"(select a from t) union all (select sum(a) from t where a > 2 group by b) order by a limit 1, 2;",
"(select a from t where a = 1) union all (select b from t where a = 2) union all (select c from t where a = 3) order by a limit 2;"
]
},
{
Expand Down
215 changes: 215 additions & 0 deletions planner/cascades/testdata/transformation_rules_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -475,6 +475,221 @@
"Group#10 Schema:[test.t.a]",
" IndexScan_8 table:t, index:c, d, e"
]
},
{
"SQL": "(select a from t) union all (select b from t) order by a limit 2;",
"Result": [
"Group#0 Schema:[Column#25]",
" TopN_26 input:[Group#1], Column#25:asc, offset:0, count:2",
"Group#1 Schema:[Column#25]",
" Union_5 input:[Group#2,Group#3]",
"Group#2 Schema:[Column#25]",
" Projection_6 input:[Group#4], test.t.a",
"Group#4 Schema:[test.t.a]",
" Projection_4 input:[Group#5], test.t.a",
"Group#5 Schema:[test.t.a]",
" TopN_29 input:[Group#6], test.t.a:asc, offset:0, count:2",
"Group#6 Schema:[test.t.a]",
" TiKVSingleGather_11 input:[Group#7], table:t",
" TiKVSingleGather_23 input:[Group#8], table:t, index:e_d_c_str_prefix",
" TiKVSingleGather_21 input:[Group#9], table:t, index:c_d_e_str",
" TiKVSingleGather_19 input:[Group#10], table:t, index:f_g",
" TiKVSingleGather_17 input:[Group#11], table:t, index:g",
" TiKVSingleGather_15 input:[Group#12], table:t, index:f",
" TiKVSingleGather_13 input:[Group#13], table:t, index:c_d_e",
"Group#7 Schema:[test.t.a]",
" TableScan_10 table:t, pk col:test.t.a",
"Group#8 Schema:[test.t.a]",
" IndexScan_22 table:t, index:e_str, d_str, c_str",
"Group#9 Schema:[test.t.a]",
" IndexScan_20 table:t, index:c_str, d_str, e_str",
"Group#10 Schema:[test.t.a]",
" IndexScan_18 table:t, index:f, g",
"Group#11 Schema:[test.t.a]",
" IndexScan_16 table:t, index:g",
"Group#12 Schema:[test.t.a]",
" IndexScan_14 table:t, index:f",
"Group#13 Schema:[test.t.a]",
" IndexScan_12 table:t, index:c, d, e",
"Group#3 Schema:[Column#25]",
" Projection_7 input:[Group#14], test.t.b",
"Group#14 Schema:[test.t.b]",
" Projection_2 input:[Group#15], test.t.b",
"Group#15 Schema:[test.t.b]",
" TopN_31 input:[Group#16], test.t.b:asc, offset:0, count:2",
"Group#16 Schema:[test.t.b]",
" TiKVSingleGather_25 input:[Group#17], table:t",
"Group#17 Schema:[test.t.b]",
" TableScan_24 table:t"
]
},
{
"SQL": "(select a from t) union all (select sum(a) from t where a > 2 group by b) order by a limit 2;",
"Result": [
"Group#0 Schema:[Column#26]",
" TopN_28 input:[Group#1], Column#26:asc, offset:0, count:2",
"Group#1 Schema:[Column#26]",
" Union_7 input:[Group#2,Group#3]",
"Group#2 Schema:[Column#26]",
" Projection_8 input:[Group#4], cast(test.t.a)->Column#26",
"Group#4 Schema:[test.t.a]",
" Projection_6 input:[Group#5], test.t.a",
"Group#5 Schema:[test.t.a]",
" TopN_31 input:[Group#6], cast(test.t.a):asc, offset:0, count:2",
"Group#6 Schema:[test.t.a]",
" TiKVSingleGather_13 input:[Group#7], table:t",
" TiKVSingleGather_25 input:[Group#8], table:t, index:e_d_c_str_prefix",
" TiKVSingleGather_23 input:[Group#9], table:t, index:c_d_e_str",
" TiKVSingleGather_21 input:[Group#10], table:t, index:f_g",
" TiKVSingleGather_19 input:[Group#11], table:t, index:g",
" TiKVSingleGather_17 input:[Group#12], table:t, index:f",
" TiKVSingleGather_15 input:[Group#13], table:t, index:c_d_e",
"Group#7 Schema:[test.t.a]",
" TableScan_12 table:t, pk col:test.t.a",
"Group#8 Schema:[test.t.a]",
" IndexScan_24 table:t, index:e_str, d_str, c_str",
"Group#9 Schema:[test.t.a]",
" IndexScan_22 table:t, index:c_str, d_str, e_str",
"Group#10 Schema:[test.t.a]",
" IndexScan_20 table:t, index:f, g",
"Group#11 Schema:[test.t.a]",
" IndexScan_18 table:t, index:g",
"Group#12 Schema:[test.t.a]",
" IndexScan_16 table:t, index:f",
"Group#13 Schema:[test.t.a]",
" IndexScan_14 table:t, index:c, d, e",
"Group#3 Schema:[Column#26]",
" Projection_9 input:[Group#14], cast(Column#13)->Column#26",
"Group#14 Schema:[Column#13]",
" Projection_4 input:[Group#15], Column#13",
"Group#15 Schema:[Column#13]",
" TopN_33 input:[Group#16], cast(Column#13):asc, offset:0, count:2",
"Group#16 Schema:[Column#13]",
" Aggregation_3 input:[Group#17], group by:test.t.b, funcs:sum(test.t.a)",
"Group#17 Schema:[test.t.a,test.t.b]",
" Selection_2 input:[Group#18], gt(test.t.a, 2)",
"Group#18 Schema:[test.t.a,test.t.b]",
" TiKVSingleGather_27 input:[Group#19], table:t",
"Group#19 Schema:[test.t.a,test.t.b]",
" TableScan_26 table:t, pk col:test.t.a"
]
},
{
"SQL": "(select a from t) union all (select sum(a) from t where a > 2 group by b) order by a limit 1, 2;",
"Result": [
"Group#0 Schema:[Column#26]",
" TopN_28 input:[Group#1], Column#26:asc, offset:1, count:2",
"Group#1 Schema:[Column#26]",
" Union_7 input:[Group#2,Group#3]",
"Group#2 Schema:[Column#26]",
" Projection_8 input:[Group#4], cast(test.t.a)->Column#26",
"Group#4 Schema:[test.t.a]",
" Projection_6 input:[Group#5], test.t.a",
"Group#5 Schema:[test.t.a]",
" TopN_31 input:[Group#6], cast(test.t.a):asc, offset:0, count:3",
"Group#6 Schema:[test.t.a]",
" TiKVSingleGather_13 input:[Group#7], table:t",
" TiKVSingleGather_25 input:[Group#8], table:t, index:e_d_c_str_prefix",
" TiKVSingleGather_23 input:[Group#9], table:t, index:c_d_e_str",
" TiKVSingleGather_21 input:[Group#10], table:t, index:f_g",
" TiKVSingleGather_19 input:[Group#11], table:t, index:g",
" TiKVSingleGather_17 input:[Group#12], table:t, index:f",
" TiKVSingleGather_15 input:[Group#13], table:t, index:c_d_e",
"Group#7 Schema:[test.t.a]",
" TableScan_12 table:t, pk col:test.t.a",
"Group#8 Schema:[test.t.a]",
" IndexScan_24 table:t, index:e_str, d_str, c_str",
"Group#9 Schema:[test.t.a]",
" IndexScan_22 table:t, index:c_str, d_str, e_str",
"Group#10 Schema:[test.t.a]",
" IndexScan_20 table:t, index:f, g",
"Group#11 Schema:[test.t.a]",
" IndexScan_18 table:t, index:g",
"Group#12 Schema:[test.t.a]",
" IndexScan_16 table:t, index:f",
"Group#13 Schema:[test.t.a]",
" IndexScan_14 table:t, index:c, d, e",
"Group#3 Schema:[Column#26]",
" Projection_9 input:[Group#14], cast(Column#13)->Column#26",
"Group#14 Schema:[Column#13]",
" Projection_4 input:[Group#15], Column#13",
"Group#15 Schema:[Column#13]",
" TopN_33 input:[Group#16], cast(Column#13):asc, offset:0, count:3",
"Group#16 Schema:[Column#13]",
" Aggregation_3 input:[Group#17], group by:test.t.b, funcs:sum(test.t.a)",
"Group#17 Schema:[test.t.a,test.t.b]",
" Selection_2 input:[Group#18], gt(test.t.a, 2)",
"Group#18 Schema:[test.t.a,test.t.b]",
" TiKVSingleGather_27 input:[Group#19], table:t",
"Group#19 Schema:[test.t.a,test.t.b]",
" TableScan_26 table:t, pk col:test.t.a"
]
},
{
"SQL": "(select a from t where a = 1) union all (select b from t where a = 2) union all (select c from t where a = 3) order by a limit 2;",
"Result": [
"Group#0 Schema:[Column#37]",
" TopN_36 input:[Group#1], Column#37:asc, offset:0, count:2",
"Group#1 Schema:[Column#37]",
" Union_10 input:[Group#2,Group#3,Group#4]",
"Group#2 Schema:[Column#37]",
" Projection_11 input:[Group#5], test.t.a",
"Group#5 Schema:[test.t.a]",
" Projection_9 input:[Group#6], test.t.a",
"Group#6 Schema:[test.t.a]",
" TopN_39 input:[Group#7], test.t.a:asc, offset:0, count:2",
"Group#7 Schema:[test.t.a]",
" Selection_8 input:[Group#8], eq(test.t.a, 1)",
"Group#8 Schema:[test.t.a]",
" TiKVSingleGather_17 input:[Group#9], table:t",
" TiKVSingleGather_29 input:[Group#10], table:t, index:e_d_c_str_prefix",
" TiKVSingleGather_27 input:[Group#11], table:t, index:c_d_e_str",
" TiKVSingleGather_25 input:[Group#12], table:t, index:f_g",
" TiKVSingleGather_23 input:[Group#13], table:t, index:g",
" TiKVSingleGather_21 input:[Group#14], table:t, index:f",
" TiKVSingleGather_19 input:[Group#15], table:t, index:c_d_e",
"Group#9 Schema:[test.t.a]",
" TableScan_16 table:t, pk col:test.t.a",
"Group#10 Schema:[test.t.a]",
" IndexScan_28 table:t, index:e_str, d_str, c_str",
"Group#11 Schema:[test.t.a]",
" IndexScan_26 table:t, index:c_str, d_str, e_str",
"Group#12 Schema:[test.t.a]",
" IndexScan_24 table:t, index:f, g",
"Group#13 Schema:[test.t.a]",
" IndexScan_22 table:t, index:g",
"Group#14 Schema:[test.t.a]",
" IndexScan_20 table:t, index:f",
"Group#15 Schema:[test.t.a]",
" IndexScan_18 table:t, index:c, d, e",
"Group#3 Schema:[Column#37]",
" Projection_12 input:[Group#16], test.t.b",
"Group#16 Schema:[test.t.b]",
" Projection_6 input:[Group#17], test.t.b",
"Group#17 Schema:[test.t.a,test.t.b]",
" TopN_41 input:[Group#18], test.t.b:asc, offset:0, count:2",
"Group#18 Schema:[test.t.a,test.t.b]",
" Selection_5 input:[Group#19], eq(test.t.a, 2)",
"Group#19 Schema:[test.t.a,test.t.b]",
" TiKVSingleGather_31 input:[Group#20], table:t",
"Group#20 Schema:[test.t.a,test.t.b]",
" TableScan_30 table:t, pk col:test.t.a",
"Group#4 Schema:[Column#37]",
" Projection_13 input:[Group#21], test.t.c",
"Group#21 Schema:[test.t.c]",
" Projection_3 input:[Group#22], test.t.c",
"Group#22 Schema:[test.t.a,test.t.c]",
" TopN_43 input:[Group#23], test.t.c:asc, offset:0, count:2",
"Group#23 Schema:[test.t.a,test.t.c]",
" Selection_2 input:[Group#24], eq(test.t.a, 3)",
"Group#24 Schema:[test.t.a,test.t.c]",
" TiKVSingleGather_33 input:[Group#25], table:t",
" TiKVSingleGather_35 input:[Group#26], table:t, index:c_d_e",
"Group#25 Schema:[test.t.a,test.t.c]",
" TableScan_32 table:t, pk col:test.t.a",
"Group#26 Schema:[test.t.a,test.t.c]",
" IndexScan_34 table:t, index:c, d, e"
]
}
]
},
Expand Down
51 changes: 51 additions & 0 deletions planner/cascades/transformation_rules.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ var defaultTransformationMap = map[memo.Operand][]Transformation{
},
memo.OperandTopN: {
NewRulePushTopNDownProjection(),
NewRulePushTopNDownUnionAll(),
},
}

Expand Down Expand Up @@ -998,6 +999,56 @@ func (r *PushTopNDownProjection) OnTransform(old *memo.ExprIter) (newExprs []*me
return []*memo.GroupExpr{projExpr}, true, false, nil
}

// PushTopNDownUnionAll pushes topN to union all.
type PushTopNDownUnionAll struct {
baseRule
}

// NewRulePushTopNDownUnionAll creates a new Transformation PushTopNDownUnionAll.
// The pattern of this rule is `TopN->UnionAll->X`.
func NewRulePushTopNDownUnionAll() Transformation {
rule := &PushTopNDownUnionAll{}
rule.pattern = memo.BuildPattern(
memo.OperandTopN,
memo.EngineTiDBOnly,
memo.NewPattern(memo.OperandUnionAll, memo.EngineTiDBOnly),
)
return rule
}

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

// OnTransform implements Transformation interface.
// It will transform `TopN->UnionAll->X` to `TopN->UnionAll->TopN->X`.
func (r *PushTopNDownUnionAll) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) {
topN := old.GetExpr().ExprNode.(*plannercore.LogicalTopN)
unionAll := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalUnionAll)

newTopN := plannercore.LogicalTopN{
Count: topN.Count + topN.Offset,
ByItems: topN.ByItems,
}.Init(topN.SCtx(), topN.SelectBlockOffset())

newUnionAllExpr := memo.NewGroupExpr(unionAll)
for _, childGroup := range old.Children[0].GetExpr().Children {
newTopNExpr := memo.NewGroupExpr(newTopN)
newTopNExpr.Children = append(newTopNExpr.Children, childGroup)
newTopNGroup := memo.NewGroupWithSchema(newTopNExpr, childGroup.Prop.Schema)

newUnionAllExpr.Children = append(newUnionAllExpr.Children, newTopNGroup)
}

newTopNExpr := memo.NewGroupExpr(topN)
newUnionAllGroup := memo.NewGroupWithSchema(newUnionAllExpr, unionAll.Schema())
newTopNExpr.SetChildren(newUnionAllGroup)
newTopNExpr.AddAppliedRule(r)
return []*memo.GroupExpr{newTopNExpr}, true, false, nil
}

// MergeAggregationProjection merges the Projection below an Aggregation as a new Aggregation.
// The Projection may be regenerated in the ImplementationPhase. But this rule allows the
// Aggregation to match other rules, such as MergeAdjacentAggregation.
Expand Down
1 change: 1 addition & 0 deletions planner/cascades/transformation_rules_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,6 +155,7 @@ func (s *testTransformationRuleSuite) TestTopNRules(c *C) {
},
memo.OperandTopN: {
NewRulePushTopNDownProjection(),
NewRulePushTopNDownUnionAll(),
},
})
var input []string
Expand Down

0 comments on commit 980f72d

Please sign in to comment.