From 56e5dd94c9713492e1bc16f20a123e994ce37c67 Mon Sep 17 00:00:00 2001 From: Feng Liyuan Date: Wed, 6 May 2020 18:41:22 +0800 Subject: [PATCH 1/9] cherry pick #16591 to release-3.0 Signed-off-by: sre-bot --- cmd/explaintest/r/explain.result | 14 + executor/aggfuncs/aggfunc_test.go | 344 ++- executor/aggfuncs/builder.go | 24 +- executor/aggfuncs/func_group_concat.go | 330 ++- executor/aggfuncs/func_group_concat_test.go | 19 +- executor/aggregate_test.go | 101 + executor/benchmark_test.go | 989 +++++++ executor/builder.go | 19 +- executor/executor_pkg_test.go | 94 + executor/executor_required_rows_test.go | 13 +- executor/sort.go | 3 +- expression/aggregation/agg_to_pb.go | 3 + expression/aggregation/aggregation.go | 28 + expression/aggregation/descriptor.go | 15 + expression/aggregation/explain.go | 21 +- planner/cascades/enforcer_rules.go | 10 +- .../transformation_rules_suite_out.json | 2356 +++++++++++++++++ planner/cascades/transformation_rules.go | 2199 +++++++++++++++ planner/core/exhaust_physical_plans.go | 5 + planner/core/explain.go | 206 ++ planner/core/find_best_task.go | 30 + planner/core/logical_plan_builder.go | 81 +- planner/core/logical_plan_test.go | 19 +- planner/core/logical_plans.go | 4 +- planner/core/pb_to_plan.go | 277 ++ planner/core/physical_plan_test.go | 757 ++++++ planner/core/physical_plans.go | 18 +- planner/core/plan.go | 7 +- planner/core/planbuilder.go | 11 + planner/core/property_cols_prune.go | 3 +- planner/core/resolve_indices.go | 6 + planner/core/rule_aggregation_push_down.go | 32 +- planner/core/rule_column_pruning.go | 48 +- planner/core/rule_inject_extra_projection.go | 87 + planner/core/rule_max_min_eliminate.go | 5 + planner/core/rule_topn_push_down.go | 8 +- planner/core/task.go | 11 + planner/core/testdata/plan_suite_in.json | 120 + planner/core/testdata/plan_suite_out.json | 682 +++++ planner/util/byitem.go | 45 + sessionctx/variable/varsutil.go | 13 +- 41 files changed, 9001 insertions(+), 56 deletions(-) create mode 100644 planner/cascades/testdata/transformation_rules_suite_out.json create mode 100644 planner/core/pb_to_plan.go create mode 100644 planner/util/byitem.go diff --git a/cmd/explaintest/r/explain.result b/cmd/explaintest/r/explain.result index 0055b8834a554..b1d11729b88ab 100644 --- a/cmd/explaintest/r/explain.result +++ b/cmd/explaintest/r/explain.result @@ -25,6 +25,7 @@ id int(11) YES NULL drop table if exists t; create table t(id int primary key, a int, b int); explain select group_concat(a) from t group by id; +<<<<<<< HEAD id count task operator info StreamAgg_8 8000.00 root group by:col_1, funcs:group_concat(col_0, ",") └─Projection_18 10000.00 root cast(test.t.a), test.t.id @@ -36,4 +37,17 @@ StreamAgg_8 8000.00 root group by:col_2, funcs:group_concat(col_0, col_1, ",") └─Projection_18 10000.00 root cast(test.t.a), cast(test.t.b), test.t.id └─TableReader_15 10000.00 root data:TableScan_14 └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:true, stats:pseudo +======= +id estRows task access object operator info +StreamAgg_8 8000.00 root group by:Column#6, funcs:group_concat(Column#5 separator ",")->Column#4 +└─Projection_18 10000.00 root cast(test.t.a, var_string(20))->Column#5, test.t.id + └─TableReader_15 10000.00 root data:TableFullScan_14 + └─TableFullScan_14 10000.00 cop[tikv] table:t keep order:true, stats:pseudo +explain select group_concat(a, b) from t group by id; +id estRows task access object operator info +StreamAgg_8 8000.00 root group by:Column#7, funcs:group_concat(Column#5, Column#6 separator ",")->Column#4 +└─Projection_18 10000.00 root cast(test.t.a, var_string(20))->Column#5, cast(test.t.b, var_string(20))->Column#6, test.t.id + └─TableReader_15 10000.00 root data:TableFullScan_14 + └─TableFullScan_14 10000.00 cop[tikv] table:t keep order:true, stats:pseudo +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) drop table t; diff --git a/executor/aggfuncs/aggfunc_test.go b/executor/aggfuncs/aggfunc_test.go index c47578975654c..487cc196d2760 100644 --- a/executor/aggfuncs/aggfunc_test.go +++ b/executor/aggfuncs/aggfunc_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/executor/aggfuncs" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" @@ -68,8 +69,22 @@ type aggTest struct { dataGen func(i int) types.Datum funcName string results []types.Datum + orderBy bool } +<<<<<<< HEAD +======= +type multiArgsAggTest struct { + dataTypes []*types.FieldType + retType *types.FieldType + numRows int + dataGens []func(i int) types.Datum + funcName string + results []types.Datum + orderBy bool +} + +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) func (s *testSuite) testMergePartialResult(c *C, p aggTest) { srcChk := chunk.NewChunkWithCapacity([]*types.FieldType{p.dataType}, p.numRows) for i := 0; i < p.numRows; i++ { @@ -84,6 +99,11 @@ func (s *testSuite) testMergePartialResult(c *C, p aggTest) { } desc, err := aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, false) c.Assert(err, IsNil) + if p.orderBy { + desc.OrderByItems = []*util.ByItems{ + {Expr: args[0], Desc: true}, + } + } partialDesc, finalDesc := desc.Split([]int{0, 1}) // build partial func for partial phase. @@ -103,7 +123,7 @@ func (s *testSuite) testMergePartialResult(c *C, p aggTest) { dt := resultChk.GetRow(0).GetDatum(0, p.dataType) result, err := dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[0]) c.Assert(err, IsNil) - c.Assert(result, Equals, 0) + c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[0])) err = finalFunc.MergePartialResult(s.ctx, partialResult, finalPr) c.Assert(err, IsNil) @@ -119,7 +139,7 @@ func (s *testSuite) testMergePartialResult(c *C, p aggTest) { dt = resultChk.GetRow(0).GetDatum(0, p.dataType) result, err = dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[1]) c.Assert(err, IsNil) - c.Assert(result, Equals, 0) + c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[1])) err = finalFunc.MergePartialResult(s.ctx, partialResult, finalPr) c.Assert(err, IsNil) @@ -130,7 +150,7 @@ func (s *testSuite) testMergePartialResult(c *C, p aggTest) { dt = resultChk.GetRow(0).GetDatum(0, p.dataType) result, err = dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[2]) c.Assert(err, IsNil) - c.Assert(result, Equals, 0) + c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[2])) } func buildAggTester(funcName string, tp byte, numRows int, results ...interface{}) aggTest { @@ -150,6 +170,107 @@ func buildAggTesterWithFieldType(funcName string, ft *types.FieldType, numRows i return pt } +<<<<<<< HEAD +======= +func (s *testSuite) testMultiArgsMergePartialResult(c *C, p multiArgsAggTest) { + srcChk := chunk.NewChunkWithCapacity(p.dataTypes, p.numRows) + for i := 0; i < p.numRows; i++ { + for j := 0; j < len(p.dataGens); j++ { + fdt := p.dataGens[j](i) + srcChk.AppendDatum(j, &fdt) + } + } + iter := chunk.NewIterator4Chunk(srcChk) + + args := make([]expression.Expression, len(p.dataTypes)) + for k := 0; k < len(p.dataTypes); k++ { + args[k] = &expression.Column{RetType: p.dataTypes[k], Index: k} + } + + desc, err := aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, false) + c.Assert(err, IsNil) + if p.orderBy { + desc.OrderByItems = []*util.ByItems{ + {Expr: args[0], Desc: true}, + } + } + partialDesc, finalDesc := desc.Split([]int{0, 1}) + + // build partial func for partial phase. + partialFunc := aggfuncs.Build(s.ctx, partialDesc, 0) + partialResult := partialFunc.AllocPartialResult() + + // build final func for final phase. + finalFunc := aggfuncs.Build(s.ctx, finalDesc, 0) + finalPr := finalFunc.AllocPartialResult() + resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{p.retType}, 1) + + // update partial result. + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + partialFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialResult) + } + partialFunc.AppendFinalResult2Chunk(s.ctx, partialResult, resultChk) + dt := resultChk.GetRow(0).GetDatum(0, p.retType) + result, err := dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[0]) + c.Assert(err, IsNil) + c.Assert(result, Equals, 0) + + err = finalFunc.MergePartialResult(s.ctx, partialResult, finalPr) + c.Assert(err, IsNil) + partialFunc.ResetPartialResult(partialResult) + + iter.Begin() + iter.Next() + for row := iter.Next(); row != iter.End(); row = iter.Next() { + partialFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialResult) + } + resultChk.Reset() + partialFunc.AppendFinalResult2Chunk(s.ctx, partialResult, resultChk) + dt = resultChk.GetRow(0).GetDatum(0, p.retType) + result, err = dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[1]) + c.Assert(err, IsNil) + c.Assert(result, Equals, 0) + err = finalFunc.MergePartialResult(s.ctx, partialResult, finalPr) + c.Assert(err, IsNil) + + resultChk.Reset() + err = finalFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) + c.Assert(err, IsNil) + + dt = resultChk.GetRow(0).GetDatum(0, p.retType) + result, err = dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[2]) + c.Assert(err, IsNil) + c.Assert(result, Equals, 0) +} + +// for multiple args in aggfuncs such as json_objectagg(c1, c2) +func buildMultiArgsAggTester(funcName string, tps []byte, rt byte, numRows int, results ...interface{}) multiArgsAggTest { + fts := make([]*types.FieldType, len(tps)) + for i := 0; i < len(tps); i++ { + fts[i] = types.NewFieldType(tps[i]) + } + return buildMultiArgsAggTesterWithFieldType(funcName, fts, types.NewFieldType(rt), numRows, results...) +} + +func buildMultiArgsAggTesterWithFieldType(funcName string, fts []*types.FieldType, rt *types.FieldType, numRows int, results ...interface{}) multiArgsAggTest { + dataGens := make([]func(i int) types.Datum, len(fts)) + for i := 0; i < len(fts); i++ { + dataGens[i] = getDataGenFunc(fts[i]) + } + mt := multiArgsAggTest{ + dataTypes: fts, + retType: rt, + numRows: numRows, + funcName: funcName, + dataGens: dataGens, + } + for _, result := range results { + mt.results = append(mt.results, types.NewDatum(result)) + } + return mt +} + +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) func getDataGenFunc(ft *types.FieldType) func(i int) types.Datum { switch ft.Tp { case mysql.TypeLonglong: @@ -198,6 +319,11 @@ func (s *testSuite) testAggFunc(c *C, p aggTest) { } desc, err := aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, false) c.Assert(err, IsNil) + if p.orderBy { + desc.OrderByItems = []*util.ByItems{ + {Expr: args[0], Desc: true}, + } + } finalFunc := aggfuncs.Build(s.ctx, desc, 0) finalPr := finalFunc.AllocPartialResult() resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{desc.RetTp}, 1) @@ -210,7 +336,7 @@ func (s *testSuite) testAggFunc(c *C, p aggTest) { dt := resultChk.GetRow(0).GetDatum(0, desc.RetTp) result, err := dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[1]) c.Assert(err, IsNil) - c.Assert(result, Equals, 0) + c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[1])) // test the empty input resultChk.Reset() @@ -219,11 +345,16 @@ func (s *testSuite) testAggFunc(c *C, p aggTest) { dt = resultChk.GetRow(0).GetDatum(0, desc.RetTp) result, err = dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[0]) c.Assert(err, IsNil) - c.Assert(result, Equals, 0) + c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[0])) // test the agg func with distinct desc, err = aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, true) c.Assert(err, IsNil) + if p.orderBy { + desc.OrderByItems = []*util.ByItems{ + {Expr: args[0], Desc: true}, + } + } finalFunc = aggfuncs.Build(s.ctx, desc, 0) finalPr = finalFunc.AllocPartialResult() @@ -239,7 +370,92 @@ func (s *testSuite) testAggFunc(c *C, p aggTest) { dt = resultChk.GetRow(0).GetDatum(0, desc.RetTp) result, err = dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[1]) c.Assert(err, IsNil) - c.Assert(result, Equals, 0) + c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[1])) + + // test the empty input + resultChk.Reset() + finalFunc.ResetPartialResult(finalPr) + finalFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) + dt = resultChk.GetRow(0).GetDatum(0, desc.RetTp) + result, err = dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[0]) + c.Assert(err, IsNil) + c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[0])) +} +<<<<<<< HEAD +======= + +func (s *testSuite) testMultiArgsAggFunc(c *C, p multiArgsAggTest) { + srcChk := chunk.NewChunkWithCapacity(p.dataTypes, p.numRows) + for i := 0; i < p.numRows; i++ { + for j := 0; j < len(p.dataGens); j++ { + fdt := p.dataGens[j](i) + srcChk.AppendDatum(j, &fdt) + } + } + srcChk.AppendDatum(0, &types.Datum{}) + + args := make([]expression.Expression, len(p.dataTypes)) + for k := 0; k < len(p.dataTypes); k++ { + args[k] = &expression.Column{RetType: p.dataTypes[k], Index: k} + } + if p.funcName == ast.AggFuncGroupConcat { + args = append(args, &expression.Constant{Value: types.NewStringDatum(" "), RetType: types.NewFieldType(mysql.TypeString)}) + } + + desc, err := aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, false) + c.Assert(err, IsNil) + if p.orderBy { + desc.OrderByItems = []*util.ByItems{ + {Expr: args[0], Desc: true}, + } + } + finalFunc := aggfuncs.Build(s.ctx, desc, 0) + finalPr := finalFunc.AllocPartialResult() + resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{desc.RetTp}, 1) + + iter := chunk.NewIterator4Chunk(srcChk) + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + finalFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, finalPr) + } + finalFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) + dt := resultChk.GetRow(0).GetDatum(0, desc.RetTp) + result, err := dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[1]) + c.Assert(err, IsNil) + c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[1])) + + // test the empty input + resultChk.Reset() + finalFunc.ResetPartialResult(finalPr) + finalFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) + dt = resultChk.GetRow(0).GetDatum(0, desc.RetTp) + result, err = dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[0]) + c.Assert(err, IsNil) + c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[0])) + + // test the agg func with distinct + desc, err = aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, true) + c.Assert(err, IsNil) + if p.orderBy { + desc.OrderByItems = []*util.ByItems{ + {Expr: args[0], Desc: true}, + } + } + finalFunc = aggfuncs.Build(s.ctx, desc, 0) + finalPr = finalFunc.AllocPartialResult() + + resultChk.Reset() + iter = chunk.NewIterator4Chunk(srcChk) + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + finalFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, finalPr) + } + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + finalFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, finalPr) + } + finalFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) + dt = resultChk.GetRow(0).GetDatum(0, desc.RetTp) + result, err = dt.CompareDatum(s.ctx.GetSessionVars().StmtCtx, &p.results[1]) + c.Assert(err, IsNil) + c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[1])) // test the empty input resultChk.Reset() @@ -250,3 +466,119 @@ func (s *testSuite) testAggFunc(c *C, p aggTest) { c.Assert(err, IsNil) c.Assert(result, Equals, 0) } + +func (s *testSuite) benchmarkAggFunc(b *testing.B, p aggTest) { + srcChk := chunk.NewChunkWithCapacity([]*types.FieldType{p.dataType}, p.numRows) + for i := 0; i < p.numRows; i++ { + dt := p.dataGen(i) + srcChk.AppendDatum(0, &dt) + } + srcChk.AppendDatum(0, &types.Datum{}) + + args := []expression.Expression{&expression.Column{RetType: p.dataType, Index: 0}} + if p.funcName == ast.AggFuncGroupConcat { + args = append(args, &expression.Constant{Value: types.NewStringDatum(" "), RetType: types.NewFieldType(mysql.TypeString)}) + } + desc, err := aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, false) + if err != nil { + b.Fatal(err) + } + if p.orderBy { + desc.OrderByItems = []*util.ByItems{ + {Expr: args[0], Desc: true}, + } + } + finalFunc := aggfuncs.Build(s.ctx, desc, 0) + resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{desc.RetTp}, 1) + iter := chunk.NewIterator4Chunk(srcChk) + input := make([]chunk.Row, 0, iter.Len()) + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + input = append(input, row) + } + b.Run(fmt.Sprintf("%v/%v", p.funcName, p.dataType), func(b *testing.B) { + s.baseBenchmarkAggFunc(b, finalFunc, input, resultChk) + }) + + desc, err = aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, true) + if err != nil { + b.Fatal(err) + } + if p.orderBy { + desc.OrderByItems = []*util.ByItems{ + {Expr: args[0], Desc: true}, + } + } + finalFunc = aggfuncs.Build(s.ctx, desc, 0) + resultChk.Reset() + b.Run(fmt.Sprintf("%v(distinct)/%v", p.funcName, p.dataType), func(b *testing.B) { + s.baseBenchmarkAggFunc(b, finalFunc, input, resultChk) + }) +} + +func (s *testSuite) benchmarkMultiArgsAggFunc(b *testing.B, p multiArgsAggTest) { + srcChk := chunk.NewChunkWithCapacity(p.dataTypes, p.numRows) + for i := 0; i < p.numRows; i++ { + for j := 0; j < len(p.dataGens); j++ { + fdt := p.dataGens[j](i) + srcChk.AppendDatum(j, &fdt) + } + } + srcChk.AppendDatum(0, &types.Datum{}) + + args := make([]expression.Expression, len(p.dataTypes)) + for k := 0; k < len(p.dataTypes); k++ { + args[k] = &expression.Column{RetType: p.dataTypes[k], Index: k} + } + if p.funcName == ast.AggFuncGroupConcat { + args = append(args, &expression.Constant{Value: types.NewStringDatum(" "), RetType: types.NewFieldType(mysql.TypeString)}) + } + + desc, err := aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, false) + if err != nil { + b.Fatal(err) + } + if p.orderBy { + desc.OrderByItems = []*util.ByItems{ + {Expr: args[0], Desc: true}, + } + } + finalFunc := aggfuncs.Build(s.ctx, desc, 0) + resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{desc.RetTp}, 1) + iter := chunk.NewIterator4Chunk(srcChk) + input := make([]chunk.Row, 0, iter.Len()) + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + input = append(input, row) + } + b.Run(fmt.Sprintf("%v/%v", p.funcName, p.dataTypes), func(b *testing.B) { + s.baseBenchmarkAggFunc(b, finalFunc, input, resultChk) + }) + + desc, err = aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, true) + if err != nil { + b.Fatal(err) + } + if p.orderBy { + desc.OrderByItems = []*util.ByItems{ + {Expr: args[0], Desc: true}, + } + } + finalFunc = aggfuncs.Build(s.ctx, desc, 0) + resultChk.Reset() + b.Run(fmt.Sprintf("%v(distinct)/%v", p.funcName, p.dataTypes), func(b *testing.B) { + s.baseBenchmarkAggFunc(b, finalFunc, input, resultChk) + }) +} + +func (s *testSuite) baseBenchmarkAggFunc(b *testing.B, + finalFunc aggfuncs.AggFunc, input []chunk.Row, output *chunk.Chunk) { + finalPr := finalFunc.AllocPartialResult() + output.Reset() + b.ResetTimer() + for i := 0; i < b.N; i++ { + finalFunc.UpdatePartialResult(s.ctx, input, finalPr) + b.StopTimer() + output.Reset() + b.StartTimer() + } +} +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) diff --git a/executor/aggfuncs/builder.go b/executor/aggfuncs/builder.go index 67502c9ea9c9e..d5449d00b35f3 100644 --- a/executor/aggfuncs/builder.go +++ b/executor/aggfuncs/builder.go @@ -294,10 +294,6 @@ func buildGroupConcat(ctx sessionctx.Context, aggFuncDesc *aggregation.AggFuncDe case aggregation.DedupMode: return nil default: - base := baseAggFunc{ - args: aggFuncDesc.Args[:len(aggFuncDesc.Args)-1], - ordinal: ordinal, - } // The last arg is promised to be a not-null string constant, so the error can be ignored. c, _ := aggFuncDesc.Args[len(aggFuncDesc.Args)-1].(*expression.Constant) sep, _, err := c.EvalString(nil, chunk.Row{}) @@ -316,10 +312,26 @@ func buildGroupConcat(ctx sessionctx.Context, aggFuncDesc *aggregation.AggFuncDe panic(fmt.Sprintf("Error happened when buildGroupConcat: %s", err.Error())) } var truncated int32 + base := baseGroupConcat4String{ + baseAggFunc: baseAggFunc{ + args: aggFuncDesc.Args[:len(aggFuncDesc.Args)-1], + ordinal: ordinal, + }, + byItems: aggFuncDesc.OrderByItems, + sep: sep, + maxLen: maxLen, + truncated: &truncated, + } if aggFuncDesc.HasDistinct { - return &groupConcatDistinct{baseGroupConcat4String{baseAggFunc: base, sep: sep, maxLen: maxLen, truncated: &truncated}} + if len(aggFuncDesc.OrderByItems) > 0 { + return &groupConcatDistinctOrder{base} + } + return &groupConcatDistinct{base} + } + if len(aggFuncDesc.OrderByItems) > 0 { + return &groupConcatOrder{base} } - return &groupConcat{baseGroupConcat4String{baseAggFunc: base, sep: sep, maxLen: maxLen, truncated: &truncated}} + return &groupConcat{base} } } diff --git a/executor/aggfuncs/func_group_concat.go b/executor/aggfuncs/func_group_concat.go index 636b2bb69006d..b0612292921f6 100644 --- a/executor/aggfuncs/func_group_concat.go +++ b/executor/aggfuncs/func_group_concat.go @@ -15,11 +15,16 @@ package aggfuncs import ( "bytes" + "container/heap" + "sort" "sync/atomic" - "github.com/cznic/mathutil" + "github.com/pingcap/parser/terror" + mysql "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/hack" @@ -28,6 +33,7 @@ import ( type baseGroupConcat4String struct { baseAggFunc + byItems []*util.ByItems sep string maxLen uint64 @@ -47,19 +53,20 @@ func (e *baseGroupConcat4String) AppendFinalResult2Chunk(sctx sessionctx.Context return nil } +func (e *baseGroupConcat4String) handleTruncateError(sctx sessionctx.Context) (err error) { + if atomic.CompareAndSwapInt32(e.truncated, 0, 1) { + if !sctx.GetSessionVars().StmtCtx.TruncateAsWarning { + return expression.ErrCutValueGroupConcat.GenWithStackByArgs(e.args[0].String()) + } + sctx.GetSessionVars().StmtCtx.AppendWarning(expression.ErrCutValueGroupConcat.GenWithStackByArgs(e.args[0].String())) + } + return nil +} + func (e *baseGroupConcat4String) truncatePartialResultIfNeed(sctx sessionctx.Context, buffer *bytes.Buffer) (err error) { if e.maxLen > 0 && uint64(buffer.Len()) > e.maxLen { - i := mathutil.MaxInt - if uint64(i) > e.maxLen { - i = int(e.maxLen) - } - buffer.Truncate(i) - if atomic.CompareAndSwapInt32(e.truncated, 0, 1) { - if !sctx.GetSessionVars().StmtCtx.TruncateAsWarning { - return expression.ErrCutValueGroupConcat.GenWithStackByArgs(e.args[0].String()) - } - sctx.GetSessionVars().StmtCtx.AppendWarning(expression.ErrCutValueGroupConcat.GenWithStackByArgs(e.args[0].String())) - } + buffer.Truncate(int(e.maxLen)) + return e.handleTruncateError(sctx) } return nil } @@ -214,3 +221,302 @@ func (e *groupConcatDistinct) SetTruncated(t *int32) { func (e *groupConcatDistinct) GetTruncated() *int32 { return e.truncated } + +type sortRow struct { + buffer *bytes.Buffer + byItems []types.Datum +} + +type topNRows struct { + rows []sortRow + desc []bool + sctx sessionctx.Context + err error + + currSize uint64 + limitSize uint64 + sepSize uint64 +} + +func (h topNRows) Len() int { + return len(h.rows) +} + +func (h topNRows) Less(i, j int) bool { + n := len(h.rows[i].byItems) + for k := 0; k < n; k++ { + ret, err := h.rows[i].byItems[k].CompareDatum(h.sctx.GetSessionVars().StmtCtx, &h.rows[j].byItems[k]) + if err != nil { + h.err = err + return false + } + if h.desc[k] { + ret = -ret + } + if ret > 0 { + return true + } + if ret < 0 { + return false + } + } + return false +} + +func (h topNRows) Swap(i, j int) { + h.rows[i], h.rows[j] = h.rows[j], h.rows[i] +} + +func (h *topNRows) Push(x interface{}) { + h.rows = append(h.rows, x.(sortRow)) +} + +func (h *topNRows) Pop() interface{} { + n := len(h.rows) + x := h.rows[n-1] + h.rows = h.rows[:n-1] + return x +} + +func (h *topNRows) tryToAdd(row sortRow) (truncated bool) { + h.currSize += uint64(row.buffer.Len()) + if len(h.rows) > 0 { + h.currSize += h.sepSize + } + heap.Push(h, row) + if h.currSize <= h.limitSize { + return false + } + + for h.currSize > h.limitSize { + debt := h.currSize - h.limitSize + if uint64(h.rows[0].buffer.Len()) > debt { + h.currSize -= debt + h.rows[0].buffer.Truncate(h.rows[0].buffer.Len() - int(debt)) + } else { + h.currSize -= uint64(h.rows[0].buffer.Len()) + h.sepSize + heap.Pop(h) + } + } + return true +} + +func (h *topNRows) reset() { + h.rows = h.rows[:0] + h.err = nil + h.currSize = 0 +} + +func (h *topNRows) concat(sep string, truncated bool) string { + buffer := new(bytes.Buffer) + sort.Sort(sort.Reverse(h)) + for i, row := range h.rows { + if i != 0 { + buffer.WriteString(sep) + } + buffer.Write(row.buffer.Bytes()) + } + if truncated && uint64(buffer.Len()) < h.limitSize { + // append the last separator, because the last separator may be truncated in tryToAdd. + buffer.WriteString(sep) + buffer.Truncate(int(h.limitSize)) + } + return buffer.String() +} + +type partialResult4GroupConcatOrder struct { + topN *topNRows +} + +type groupConcatOrder struct { + baseGroupConcat4String +} + +func (e *groupConcatOrder) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { + p := (*partialResult4GroupConcatOrder)(pr) + if p.topN.Len() == 0 { + chk.AppendNull(e.ordinal) + return nil + } + chk.AppendString(e.ordinal, p.topN.concat(e.sep, *e.truncated == 1)) + return nil +} + +func (e *groupConcatOrder) AllocPartialResult() PartialResult { + desc := make([]bool, len(e.byItems)) + for i, byItem := range e.byItems { + desc[i] = byItem.Desc + } + p := &partialResult4GroupConcatOrder{ + topN: &topNRows{ + desc: desc, + currSize: 0, + limitSize: e.maxLen, + sepSize: uint64(len(e.sep)), + }, + } + return PartialResult(p) +} + +func (e *groupConcatOrder) ResetPartialResult(pr PartialResult) { + p := (*partialResult4GroupConcatOrder)(pr) + p.topN.reset() +} + +func (e *groupConcatOrder) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) (err error) { + p := (*partialResult4GroupConcatOrder)(pr) + p.topN.sctx = sctx + v, isNull := "", false + for _, row := range rowsInGroup { + buffer := new(bytes.Buffer) + for _, arg := range e.args { + v, isNull, err = arg.EvalString(sctx, row) + if err != nil { + return err + } + if isNull { + break + } + buffer.WriteString(v) + } + if isNull { + continue + } + sortRow := sortRow{ + buffer: buffer, + byItems: make([]types.Datum, 0, len(e.byItems)), + } + for _, byItem := range e.byItems { + d, err := byItem.Expr.Eval(row) + if err != nil { + return err + } + sortRow.byItems = append(sortRow.byItems, d) + } + truncated := p.topN.tryToAdd(sortRow) + if p.topN.err != nil { + return p.topN.err + } + if truncated { + if err := e.handleTruncateError(sctx); err != nil { + return err + } + } + } + return nil +} + +func (e *groupConcatOrder) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + // If order by exists, the parallel hash aggregation is forbidden in executorBuilder.buildHashAgg. + // So MergePartialResult will not be called. + return terror.ClassOptimizer.New(mysql.ErrInternal, mysql.MySQLErrName[mysql.ErrInternal]).GenWithStack("groupConcatOrder.MergePartialResult should not be called") +} + +// SetTruncated will be called in `executorBuilder#buildHashAgg` with duck-type. +func (e *groupConcatOrder) SetTruncated(t *int32) { + e.truncated = t +} + +// GetTruncated will be called in `executorBuilder#buildHashAgg` with duck-type. +func (e *groupConcatOrder) GetTruncated() *int32 { + return e.truncated +} + +type partialResult4GroupConcatOrderDistinct struct { + topN *topNRows + valSet set.StringSet + encodeBytesBuffer []byte +} + +type groupConcatDistinctOrder struct { + baseGroupConcat4String +} + +func (e *groupConcatDistinctOrder) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { + p := (*partialResult4GroupConcatOrderDistinct)(pr) + if p.topN.Len() == 0 { + chk.AppendNull(e.ordinal) + return nil + } + chk.AppendString(e.ordinal, p.topN.concat(e.sep, *e.truncated == 1)) + return nil +} + +func (e *groupConcatDistinctOrder) AllocPartialResult() PartialResult { + desc := make([]bool, len(e.byItems)) + for i, byItem := range e.byItems { + desc[i] = byItem.Desc + } + p := &partialResult4GroupConcatOrderDistinct{ + topN: &topNRows{ + desc: desc, + currSize: 0, + limitSize: e.maxLen, + sepSize: uint64(len(e.sep)), + }, + valSet: set.NewStringSet(), + } + return PartialResult(p) +} + +func (e *groupConcatDistinctOrder) ResetPartialResult(pr PartialResult) { + p := (*partialResult4GroupConcatOrderDistinct)(pr) + p.topN.reset() + p.valSet = set.NewStringSet() +} + +func (e *groupConcatDistinctOrder) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) (err error) { + p := (*partialResult4GroupConcatOrderDistinct)(pr) + p.topN.sctx = sctx + v, isNull := "", false + for _, row := range rowsInGroup { + buffer := new(bytes.Buffer) + p.encodeBytesBuffer = p.encodeBytesBuffer[:0] + for _, arg := range e.args { + v, isNull, err = arg.EvalString(sctx, row) + if err != nil { + return err + } + if isNull { + break + } + p.encodeBytesBuffer = codec.EncodeBytes(p.encodeBytesBuffer, hack.Slice(v)) + buffer.WriteString(v) + } + if isNull { + continue + } + joinedVal := string(p.encodeBytesBuffer) + if p.valSet.Exist(joinedVal) { + continue + } + p.valSet.Insert(joinedVal) + sortRow := sortRow{ + buffer: buffer, + byItems: make([]types.Datum, 0, len(e.byItems)), + } + for _, byItem := range e.byItems { + d, err := byItem.Expr.Eval(row) + if err != nil { + return err + } + sortRow.byItems = append(sortRow.byItems, d) + } + truncated := p.topN.tryToAdd(sortRow) + if p.topN.err != nil { + return p.topN.err + } + if truncated { + if err := e.handleTruncateError(sctx); err != nil { + return err + } + } + } + return nil +} + +func (e *groupConcatDistinctOrder) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + // If order by exists, the parallel hash aggregation is forbidden in executorBuilder.buildHashAgg. + // So MergePartialResult will not be called. + return terror.ClassOptimizer.New(mysql.ErrInternal, mysql.MySQLErrName[mysql.ErrInternal]).GenWithStack("groupConcatDistinctOrder.MergePartialResult should not be called") +} diff --git a/executor/aggfuncs/func_group_concat_test.go b/executor/aggfuncs/func_group_concat_test.go index 576ed8ffd2ad9..7e68e93cfed3b 100644 --- a/executor/aggfuncs/func_group_concat_test.go +++ b/executor/aggfuncs/func_group_concat_test.go @@ -14,9 +14,13 @@ package aggfuncs_test import ( + "fmt" + . "github.com/pingcap/check" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/types" ) func (s *testSuite) TestMergePartialResult4GroupConcat(c *C) { @@ -25,6 +29,19 @@ func (s *testSuite) TestMergePartialResult4GroupConcat(c *C) { } func (s *testSuite) TestGroupConcat(c *C) { - test := buildAggTester(ast.AggFuncGroupConcat, mysql.TypeString, 5, nil, "0 1 2 3 4", "0 1 2 3 4 2 3 4") + test := buildAggTester(ast.AggFuncGroupConcat, mysql.TypeString, 5, nil, "0 1 2 3 4") s.testAggFunc(c, test) + + test2 := buildMultiArgsAggTester(ast.AggFuncGroupConcat, []byte{mysql.TypeString, mysql.TypeString}, mysql.TypeString, 5, nil, "44 33 22 11 00") + test2.orderBy = true + s.testMultiArgsAggFunc(c, test2) + + defer variable.SetSessionSystemVar(s.ctx.GetSessionVars(), variable.GroupConcatMaxLen, types.NewStringDatum("1024")) + // minimum GroupConcatMaxLen is 4 + for i := 4; i <= 7; i++ { + variable.SetSessionSystemVar(s.ctx.GetSessionVars(), variable.GroupConcatMaxLen, types.NewStringDatum(fmt.Sprint(i))) + test2 = buildMultiArgsAggTester(ast.AggFuncGroupConcat, []byte{mysql.TypeString, mysql.TypeString}, mysql.TypeString, 5, nil, "44 33 22 11 00"[:i]) + test2.orderBy = true + s.testMultiArgsAggFunc(c, test2) + } } diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index d2a64108354fb..b3345a8b220c1 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -14,6 +14,8 @@ package executor_test import ( + "fmt" + . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/parser/terror" @@ -397,10 +399,16 @@ func (s *testSuite1) TestAggPrune(c *C) { tk.MustQuery("SELECT a, MIN(b), MAX(b) FROM t GROUP BY a").Check(testkit.Rows("1 11 11", "3 ")) } +<<<<<<< HEAD func (s *testSuite1) TestGroupConcatAggr(c *C) { +======= +func (s *testSuiteAgg) TestGroupConcatAggr(c *C) { + var err error +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) // issue #5411 tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") + tk.MustExec("drop table if exists test;") tk.MustExec("create table test(id int, name int)") tk.MustExec("insert into test values(1, 10);") tk.MustExec("insert into test values(1, 20);") @@ -426,6 +434,99 @@ func (s *testSuite1) TestGroupConcatAggr(c *C) { result = tk.MustQuery("select id, group_concat(name SEPARATOR '123') from test group by id order by id") result.Check(testkit.Rows("1 101232012330", "2 20", "3 200123500")) + tk.MustQuery("select group_concat(id ORDER BY name) from (select * from test order by id, name limit 2,2) t").Check(testkit.Rows("2,1")) + tk.MustQuery("select group_concat(id ORDER BY name desc) from (select * from test order by id, name limit 2,2) t").Check(testkit.Rows("1,2")) + tk.MustQuery("select group_concat(name ORDER BY id) from (select * from test order by id, name limit 2,2) t").Check(testkit.Rows("30,20")) + tk.MustQuery("select group_concat(name ORDER BY id desc) from (select * from test order by id, name limit 2,2) t").Check(testkit.Rows("20,30")) + + result = tk.MustQuery("select group_concat(name ORDER BY name desc SEPARATOR '++') from test;") + result.Check(testkit.Rows("500++200++30++20++20++10")) + + result = tk.MustQuery("select group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from test;") + result.Check(testkit.Rows("3--3--1--1--2--1")) + + result = tk.MustQuery("select group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from test;") + result.Check(testkit.Rows("500++200++30++20++20++10 3--3--1--1--2--1")) + + result = tk.MustQuery("select group_concat(distinct name order by name desc) from test;") + result.Check(testkit.Rows("500,200,30,20,10")) + + expected := "3--3--1--1--2--1" + for maxLen := 4; maxLen < len(expected); maxLen++ { + tk.MustExec(fmt.Sprintf("set session group_concat_max_len=%v", maxLen)) + result = tk.MustQuery("select group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from test;") + result.Check(testkit.Rows(expected[:maxLen])) + c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) + } + expected = "1--2--1--1--3--3" + for maxLen := 4; maxLen < len(expected); maxLen++ { + tk.MustExec(fmt.Sprintf("set session group_concat_max_len=%v", maxLen)) + result = tk.MustQuery("select group_concat(id ORDER BY name asc, id desc SEPARATOR '--') from test;") + result.Check(testkit.Rows(expected[:maxLen])) + c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) + } + expected = "500,200,30,20,10" + for maxLen := 4; maxLen < len(expected); maxLen++ { + tk.MustExec(fmt.Sprintf("set session group_concat_max_len=%v", maxLen)) + result = tk.MustQuery("select group_concat(distinct name order by name desc) from test;") + result.Check(testkit.Rows(expected[:maxLen])) + c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) + } + + tk.MustExec(fmt.Sprintf("set session group_concat_max_len=%v", 1024)) + + // test varchar table + tk.MustExec("drop table if exists test2;") + tk.MustExec("create table test2(id varchar(20), name varchar(20));") + tk.MustExec("insert into test2 select * from test;") + + tk.MustQuery("select group_concat(id ORDER BY name) from (select * from test2 order by id, name limit 2,2) t").Check(testkit.Rows("2,1")) + tk.MustQuery("select group_concat(id ORDER BY name desc) from (select * from test2 order by id, name limit 2,2) t").Check(testkit.Rows("1,2")) + tk.MustQuery("select group_concat(name ORDER BY id) from (select * from test2 order by id, name limit 2,2) t").Check(testkit.Rows("30,20")) + tk.MustQuery("select group_concat(name ORDER BY id desc) from (select * from test2 order by id, name limit 2,2) t").Check(testkit.Rows("20,30")) + + result = tk.MustQuery("select group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from test2;") + result.Check(testkit.Rows("500++30++200++20++20++10 3--1--3--1--2--1")) + + // test Position Expr + tk.MustQuery("select 1, 2, 3, 4, 5 , group_concat(name, id ORDER BY 1 desc, id SEPARATOR '++') from test;").Check(testkit.Rows("1 2 3 4 5 5003++2003++301++201++202++101")) + tk.MustQuery("select 1, 2, 3, 4, 5 , group_concat(name, id ORDER BY 2 desc, name SEPARATOR '++') from test;").Check(testkit.Rows("1 2 3 4 5 2003++5003++202++101++201++301")) + err = tk.ExecToErr("select 1, 2, 3, 4, 5 , group_concat(name, id ORDER BY 3 desc, name SEPARATOR '++') from test;") + c.Assert(err.Error(), Equals, "[planner:1054]Unknown column '3' in 'order clause'") + + // test Param Marker + tk.MustExec(`prepare s1 from "select 1, 2, 3, 4, 5 , group_concat(name, id ORDER BY floor(id/?) desc, name SEPARATOR '++') from test";`) + tk.MustExec("set @a=2;") + tk.MustQuery("execute s1 using @a;").Check(testkit.Rows("1 2 3 4 5 202++2003++5003++101++201++301")) + + tk.MustExec(`prepare s1 from "select 1, 2, 3, 4, 5 , group_concat(name, id ORDER BY ? desc, name SEPARATOR '++') from test";`) + tk.MustExec("set @a=2;") + tk.MustQuery("execute s1 using @a;").Check(testkit.Rows("1 2 3 4 5 2003++5003++202++101++201++301")) + tk.MustExec("set @a=3;") + err = tk.ExecToErr("execute s1 using @a;") + c.Assert(err.Error(), Equals, "[planner:1054]Unknown column '?' in 'order clause'") + tk.MustExec("set @a=3.0;") + tk.MustQuery("execute s1 using @a;").Check(testkit.Rows("1 2 3 4 5 101++202++201++301++2003++5003")) + + // test partition table + tk.MustExec("drop table if exists ptest;") + tk.MustExec("CREATE TABLE ptest (id int,name int) PARTITION BY RANGE ( id ) " + + "(PARTITION `p0` VALUES LESS THAN (2), PARTITION `p1` VALUES LESS THAN (11))") + tk.MustExec("insert into ptest select * from test;") + + for i := 0; i <= 1; i++ { + for j := 0; j <= 1; j++ { + tk.MustExec(fmt.Sprintf("set session tidb_opt_distinct_agg_push_down = %v", i)) + tk.MustExec(fmt.Sprintf("set session tidb_opt_agg_push_down = %v", j)) + + result = tk.MustQuery("select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from ptest;") + result.Check(testkit.Rows("500++200++30++20++20++10 3--3--1--1--2--1")) + + result = tk.MustQuery("select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from ptest;") + result.Check(testkit.Rows("500,200,30,20,10")) + } + } + // issue #9920 tk.MustQuery("select group_concat(123, null)").Check(testkit.Rows("")) } diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index aff3cd4e375a3..3015868a9d4e2 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" @@ -375,7 +376,30 @@ func buildWindowExecutor(ctx sessionctx.Context, windowFunc string, src Executor desc, _ := aggregation.NewWindowFuncDesc(ctx, windowFunc, args) plan.WindowFuncDescs = []*aggregation.WindowFuncDesc{desc} for _, col := range partitionBy { +<<<<<<< HEAD plan.PartitionBy = append(plan.PartitionBy, property.Item{Col: col}) +======= + win.PartitionBy = append(win.PartitionBy, property.Item{Col: col}) + } + win.Frame = frame + win.OrderBy = nil + + win.SetSchema(winSchema) + win.Init(ctx, nil, 0) + + var tail core.PhysicalPlan = win + if !dataSourceSorted { + byItems := make([]*util.ByItems, 0, len(partitionBy)) + for _, col := range partitionBy { + byItems = append(byItems, &util.ByItems{Expr: col, Desc: false}) + } + sort := &core.PhysicalSort{ByItems: byItems} + sort.SetChildren(src) + win.SetChildren(sort) + tail = sort + } else { + win.SetChildren(src) +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } plan.OrderBy = nil plan.SetSchema(schema) @@ -503,3 +527,968 @@ func BenchmarkWindowFunctions(b *testing.B) { }) } } +<<<<<<< HEAD +======= + +func BenchmarkWindowFunctionsWithSlidingWindow(b *testing.B) { + baseBenchmarkWindowFunctionsWithSlidingWindow(b, ast.Rows) + baseBenchmarkWindowFunctionsWithSlidingWindow(b, ast.Ranges) +} + +type hashJoinTestCase struct { + rows int + cols []*types.FieldType + concurrency int + ctx sessionctx.Context + keyIdx []int + joinType core.JoinType + disk bool + useOuterToBuild bool + rawData string + childrenUsedSchema [][]bool +} + +func (tc hashJoinTestCase) columns() []*expression.Column { + ret := make([]*expression.Column, 0) + for i, t := range tc.cols { + column := &expression.Column{Index: i, RetType: t, UniqueID: tc.ctx.GetSessionVars().AllocPlanColumnID()} + ret = append(ret, column) + } + return ret +} + +func (tc hashJoinTestCase) String() string { + return fmt.Sprintf("(rows:%v, cols:%v, concurency:%v, joinKeyIdx: %v, disk:%v)", + tc.rows, tc.cols, tc.concurrency, tc.keyIdx, tc.disk) +} + +func defaultHashJoinTestCase(cols []*types.FieldType, joinType core.JoinType, useOuterToBuild bool) *hashJoinTestCase { + ctx := mock.NewContext() + ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, -1) + ctx.GetSessionVars().StmtCtx.DiskTracker = disk.NewTracker(nil, -1) + ctx.GetSessionVars().IndexLookupJoinConcurrency = 4 + tc := &hashJoinTestCase{rows: 100000, concurrency: 4, ctx: ctx, keyIdx: []int{0, 1}, rawData: wideString} + tc.cols = cols + tc.useOuterToBuild = useOuterToBuild + tc.joinType = joinType + return tc +} + +func prepare4HashJoin(testCase *hashJoinTestCase, innerExec, outerExec Executor) *HashJoinExec { + if testCase.useOuterToBuild { + innerExec, outerExec = outerExec, innerExec + } + cols0 := innerExec.Schema().Columns + cols1 := outerExec.Schema().Columns + + joinSchema := expression.NewSchema() + if testCase.childrenUsedSchema != nil { + for i, used := range testCase.childrenUsedSchema[0] { + if used { + joinSchema.Append(cols0[i]) + } + } + for i, used := range testCase.childrenUsedSchema[1] { + if used { + joinSchema.Append(cols1[i]) + } + } + } else { + joinSchema.Append(cols0...) + joinSchema.Append(cols1...) + } + + joinKeys := make([]*expression.Column, 0, len(testCase.keyIdx)) + for _, keyIdx := range testCase.keyIdx { + joinKeys = append(joinKeys, cols0[keyIdx]) + } + probeKeys := make([]*expression.Column, 0, len(testCase.keyIdx)) + for _, keyIdx := range testCase.keyIdx { + probeKeys = append(probeKeys, cols1[keyIdx]) + } + e := &HashJoinExec{ + baseExecutor: newBaseExecutor(testCase.ctx, joinSchema, stringutil.StringerStr("HashJoin"), innerExec, outerExec), + concurrency: uint(testCase.concurrency), + joinType: testCase.joinType, // 0 for InnerJoin, 1 for LeftOutersJoin, 2 for RightOuterJoin + isOuterJoin: false, + buildKeys: joinKeys, + probeKeys: probeKeys, + buildSideExec: innerExec, + probeSideExec: outerExec, + buildSideEstCount: float64(testCase.rows), + useOuterToBuild: testCase.useOuterToBuild, + } + + childrenUsedSchema := markChildrenUsedCols(e.Schema(), e.children[0].Schema(), e.children[1].Schema()) + defaultValues := make([]types.Datum, e.buildSideExec.Schema().Len()) + lhsTypes, rhsTypes := retTypes(innerExec), retTypes(outerExec) + e.joiners = make([]joiner, e.concurrency) + for i := uint(0); i < e.concurrency; i++ { + e.joiners[i] = newJoiner(testCase.ctx, e.joinType, true, defaultValues, + nil, lhsTypes, rhsTypes, childrenUsedSchema) + } + memLimit := int64(-1) + if testCase.disk { + memLimit = 1 + } + t := memory.NewTracker(stringutil.StringerStr("root of prepare4HashJoin"), memLimit) + t.SetActionOnExceed(nil) + t2 := disk.NewTracker(stringutil.StringerStr("root of prepare4HashJoin"), -1) + e.ctx.GetSessionVars().StmtCtx.MemTracker = t + e.ctx.GetSessionVars().StmtCtx.DiskTracker = t2 + return e +} + +func benchmarkHashJoinExecWithCase(b *testing.B, casTest *hashJoinTestCase) { + opt1 := mockDataSourceParameters{ + rows: casTest.rows, + ctx: casTest.ctx, + genDataFunc: func(row int, typ *types.FieldType) interface{} { + switch typ.Tp { + case mysql.TypeLong, mysql.TypeLonglong: + return int64(row) + case mysql.TypeVarString: + return casTest.rawData + case mysql.TypeDouble: + return float64(row) + default: + panic("not implement") + } + }, + } + opt2 := opt1 + opt1.schema = expression.NewSchema(casTest.columns()...) + opt2.schema = expression.NewSchema(casTest.columns()...) + dataSource1 := buildMockDataSource(opt1) + dataSource2 := buildMockDataSource(opt2) + + b.ResetTimer() + for i := 0; i < b.N; i++ { + b.StopTimer() + exec := prepare4HashJoin(casTest, dataSource1, dataSource2) + tmpCtx := context.Background() + chk := newFirstChunk(exec) + dataSource1.prepareChunks() + dataSource2.prepareChunks() + + totalRow := 0 + b.StartTimer() + if err := exec.Open(tmpCtx); err != nil { + b.Fatal(err) + } + for { + if err := exec.Next(tmpCtx, chk); err != nil { + b.Fatal(err) + } + if chk.NumRows() == 0 { + break + } + totalRow += chk.NumRows() + } + + if spilled := exec.rowContainer.alreadySpilled(); spilled != casTest.disk { + b.Fatal("wrong usage with disk:", spilled, casTest.disk) + } + if err := exec.Close(); err != nil { + b.Fatal(err) + } + b.StopTimer() + if totalRow == 0 { + b.Fatal("totalRow == 0") + } + } +} + +func BenchmarkHashJoinInlineProjection(b *testing.B) { + cols := []*types.FieldType{ + types.NewFieldType(mysql.TypeLonglong), + types.NewFieldType(mysql.TypeVarString), + } + + b.ReportAllocs() + + { + cas := defaultHashJoinTestCase(cols, 0, false) + cas.keyIdx = []int{0} + cas.childrenUsedSchema = [][]bool{ + {false, true}, + {false, false}, + } + b.Run("InlineProjection:ON", func(b *testing.B) { + benchmarkHashJoinExecWithCase(b, cas) + }) + } + + { + cas := defaultHashJoinTestCase(cols, 0, false) + cas.keyIdx = []int{0} + b.Run("InlineProjection:OFF", func(b *testing.B) { + benchmarkHashJoinExecWithCase(b, cas) + }) + } +} + +func BenchmarkHashJoinExec(b *testing.B) { + lvl := log.GetLevel() + log.SetLevel(zapcore.ErrorLevel) + defer log.SetLevel(lvl) + + cols := []*types.FieldType{ + types.NewFieldType(mysql.TypeLonglong), + types.NewFieldType(mysql.TypeVarString), + } + + b.ReportAllocs() + cas := defaultHashJoinTestCase(cols, 0, false) + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkHashJoinExecWithCase(b, cas) + }) + + cas.keyIdx = []int{0} + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkHashJoinExecWithCase(b, cas) + }) + + cas.keyIdx = []int{0} + cas.disk = true + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkHashJoinExecWithCase(b, cas) + }) + + // Replace the wide string column with double column + cols = []*types.FieldType{ + types.NewFieldType(mysql.TypeLonglong), + types.NewFieldType(mysql.TypeDouble), + } + + cas = defaultHashJoinTestCase(cols, 0, false) + cas.keyIdx = []int{0} + cas.rows = 5 + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkHashJoinExecWithCase(b, cas) + }) + + cas = defaultHashJoinTestCase(cols, 0, false) + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkHashJoinExecWithCase(b, cas) + }) + + cas.keyIdx = []int{0} + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkHashJoinExecWithCase(b, cas) + }) +} + +func BenchmarkOuterHashJoinExec(b *testing.B) { + lvl := log.GetLevel() + log.SetLevel(zapcore.ErrorLevel) + defer log.SetLevel(lvl) + + cols := []*types.FieldType{ + types.NewFieldType(mysql.TypeLonglong), + types.NewFieldType(mysql.TypeVarString), + } + + b.ReportAllocs() + cas := defaultHashJoinTestCase(cols, 2, true) + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkHashJoinExecWithCase(b, cas) + }) + + cas.keyIdx = []int{0} + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkHashJoinExecWithCase(b, cas) + }) + + cas.keyIdx = []int{0} + cas.disk = true + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkHashJoinExecWithCase(b, cas) + }) + + // Replace the wide string column with double column + cols = []*types.FieldType{ + types.NewFieldType(mysql.TypeLonglong), + types.NewFieldType(mysql.TypeDouble), + } + + cas = defaultHashJoinTestCase(cols, 2, true) + cas.keyIdx = []int{0} + cas.rows = 5 + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkHashJoinExecWithCase(b, cas) + }) + + cas = defaultHashJoinTestCase(cols, 2, true) + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkHashJoinExecWithCase(b, cas) + }) + + cas.keyIdx = []int{0} + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkHashJoinExecWithCase(b, cas) + }) +} + +func benchmarkBuildHashTableForList(b *testing.B, casTest *hashJoinTestCase) { + opt := mockDataSourceParameters{ + schema: expression.NewSchema(casTest.columns()...), + rows: casTest.rows, + ctx: casTest.ctx, + genDataFunc: func(row int, typ *types.FieldType) interface{} { + switch typ.Tp { + case mysql.TypeLong, mysql.TypeLonglong: + return int64(row) + case mysql.TypeVarString: + return casTest.rawData + default: + panic("not implement") + } + }, + } + dataSource1 := buildMockDataSource(opt) + dataSource2 := buildMockDataSource(opt) + + dataSource1.prepareChunks() + b.ResetTimer() + for i := 0; i < b.N; i++ { + b.StopTimer() + exec := prepare4HashJoin(casTest, dataSource1, dataSource2) + tmpCtx := context.Background() + if err := exec.Open(tmpCtx); err != nil { + b.Fatal(err) + } + exec.prepared = true + + innerResultCh := make(chan *chunk.Chunk, len(dataSource1.chunks)) + for _, chk := range dataSource1.chunks { + innerResultCh <- chk + } + close(innerResultCh) + + b.StartTimer() + if err := exec.buildHashTableForList(innerResultCh); err != nil { + b.Fatal(err) + } + + if err := exec.Close(); err != nil { + b.Fatal(err) + } + b.StopTimer() + if exec.rowContainer.alreadySpilled() != casTest.disk { + b.Fatal("wrong usage with disk") + } + } +} + +func BenchmarkBuildHashTableForList(b *testing.B) { + lvl := log.GetLevel() + log.SetLevel(zapcore.ErrorLevel) + defer log.SetLevel(lvl) + + cols := []*types.FieldType{ + types.NewFieldType(mysql.TypeLonglong), + types.NewFieldType(mysql.TypeVarString), + } + + b.ReportAllocs() + cas := defaultHashJoinTestCase(cols, 0, false) + rows := []int{10, 100000} + keyIdxs := [][]int{{0, 1}, {0}} + disks := []bool{false, true} + for _, row := range rows { + for _, keyIdx := range keyIdxs { + for _, disk := range disks { + cas.rows = row + cas.keyIdx = keyIdx + cas.disk = disk + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkBuildHashTableForList(b, cas) + }) + } + } + } +} + +type indexJoinTestCase struct { + outerRows int + innerRows int + concurrency int + ctx sessionctx.Context + outerJoinKeyIdx []int + innerJoinKeyIdx []int + innerIdx []int + needOuterSort bool + rawData string +} + +func (tc indexJoinTestCase) columns() []*expression.Column { + return []*expression.Column{ + {Index: 0, RetType: types.NewFieldType(mysql.TypeLonglong)}, + {Index: 1, RetType: types.NewFieldType(mysql.TypeDouble)}, + {Index: 2, RetType: types.NewFieldType(mysql.TypeVarString)}, + } +} + +func defaultIndexJoinTestCase() *indexJoinTestCase { + ctx := mock.NewContext() + ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().SnapshotTS = 1 + ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, -1) + ctx.GetSessionVars().StmtCtx.DiskTracker = disk.NewTracker(nil, -1) + tc := &indexJoinTestCase{ + outerRows: 100000, + innerRows: variable.DefMaxChunkSize * 100, + concurrency: 4, + ctx: ctx, + outerJoinKeyIdx: []int{0, 1}, + innerJoinKeyIdx: []int{0, 1}, + innerIdx: []int{0, 1}, + rawData: wideString, + } + return tc +} + +func (tc indexJoinTestCase) String() string { + return fmt.Sprintf("(outerRows:%v, innerRows:%v, concurency:%v, outerJoinKeyIdx: %v, innerJoinKeyIdx: %v, NeedOuterSort:%v)", + tc.outerRows, tc.innerRows, tc.concurrency, tc.outerJoinKeyIdx, tc.innerJoinKeyIdx, tc.needOuterSort) +} +func (tc indexJoinTestCase) getMockDataSourceOptByRows(rows int) mockDataSourceParameters { + return mockDataSourceParameters{ + schema: expression.NewSchema(tc.columns()...), + rows: rows, + ctx: tc.ctx, + genDataFunc: func(row int, typ *types.FieldType) interface{} { + switch typ.Tp { + case mysql.TypeLong, mysql.TypeLonglong: + return int64(row) + case mysql.TypeDouble: + return float64(row) + case mysql.TypeVarString: + return tc.rawData + default: + panic("not implement") + } + }, + } +} + +func prepare4IndexInnerHashJoin(tc *indexJoinTestCase, outerDS *mockDataSource, innerDS *mockDataSource) Executor { + outerCols, innerCols := tc.columns(), tc.columns() + joinSchema := expression.NewSchema(outerCols...) + joinSchema.Append(innerCols...) + leftTypes, rightTypes := retTypes(outerDS), retTypes(innerDS) + defaultValues := make([]types.Datum, len(innerCols)) + colLens := make([]int, len(innerCols)) + for i := range colLens { + colLens[i] = types.UnspecifiedLength + } + keyOff2IdxOff := make([]int, len(tc.outerJoinKeyIdx)) + for i := range keyOff2IdxOff { + keyOff2IdxOff[i] = i + } + e := &IndexLookUpJoin{ + baseExecutor: newBaseExecutor(tc.ctx, joinSchema, stringutil.StringerStr("IndexInnerHashJoin"), outerDS), + outerCtx: outerCtx{ + rowTypes: leftTypes, + keyCols: tc.outerJoinKeyIdx, + }, + innerCtx: innerCtx{ + readerBuilder: &dataReaderBuilder{Plan: &mockPhysicalIndexReader{e: innerDS}, executorBuilder: newExecutorBuilder(tc.ctx, nil)}, + rowTypes: rightTypes, + colLens: colLens, + keyCols: tc.innerJoinKeyIdx, + }, + workerWg: new(sync.WaitGroup), + joiner: newJoiner(tc.ctx, 0, false, defaultValues, nil, leftTypes, rightTypes, nil), + isOuterJoin: false, + keyOff2IdxOff: keyOff2IdxOff, + lastColHelper: nil, + } + e.joinResult = newFirstChunk(e) + return e +} + +func prepare4IndexOuterHashJoin(tc *indexJoinTestCase, outerDS *mockDataSource, innerDS *mockDataSource) Executor { + e := prepare4IndexInnerHashJoin(tc, outerDS, innerDS).(*IndexLookUpJoin) + idxHash := &IndexNestedLoopHashJoin{IndexLookUpJoin: *e} + concurrency := tc.concurrency + idxHash.joiners = make([]joiner, concurrency) + for i := 0; i < concurrency; i++ { + idxHash.joiners[i] = e.joiner.Clone() + } + return idxHash +} + +func prepare4IndexMergeJoin(tc *indexJoinTestCase, outerDS *mockDataSource, innerDS *mockDataSource) Executor { + outerCols, innerCols := tc.columns(), tc.columns() + joinSchema := expression.NewSchema(outerCols...) + joinSchema.Append(innerCols...) + outerJoinKeys := make([]*expression.Column, 0, len(tc.outerJoinKeyIdx)) + innerJoinKeys := make([]*expression.Column, 0, len(tc.innerJoinKeyIdx)) + for _, keyIdx := range tc.outerJoinKeyIdx { + outerJoinKeys = append(outerJoinKeys, outerCols[keyIdx]) + } + for _, keyIdx := range tc.innerJoinKeyIdx { + innerJoinKeys = append(innerJoinKeys, innerCols[keyIdx]) + } + leftTypes, rightTypes := retTypes(outerDS), retTypes(innerDS) + defaultValues := make([]types.Datum, len(innerCols)) + colLens := make([]int, len(innerCols)) + for i := range colLens { + colLens[i] = types.UnspecifiedLength + } + keyOff2IdxOff := make([]int, len(outerJoinKeys)) + for i := range keyOff2IdxOff { + keyOff2IdxOff[i] = i + } + + compareFuncs := make([]expression.CompareFunc, 0, len(outerJoinKeys)) + outerCompareFuncs := make([]expression.CompareFunc, 0, len(outerJoinKeys)) + for i := range outerJoinKeys { + compareFuncs = append(compareFuncs, expression.GetCmpFunction(nil, outerJoinKeys[i], innerJoinKeys[i])) + outerCompareFuncs = append(outerCompareFuncs, expression.GetCmpFunction(nil, outerJoinKeys[i], outerJoinKeys[i])) + } + e := &IndexLookUpMergeJoin{ + baseExecutor: newBaseExecutor(tc.ctx, joinSchema, stringutil.StringerStr("IndexMergeJoin"), outerDS), + outerMergeCtx: outerMergeCtx{ + rowTypes: leftTypes, + keyCols: tc.outerJoinKeyIdx, + joinKeys: outerJoinKeys, + needOuterSort: tc.needOuterSort, + compareFuncs: outerCompareFuncs, + }, + innerMergeCtx: innerMergeCtx{ + readerBuilder: &dataReaderBuilder{Plan: &mockPhysicalIndexReader{e: innerDS}, executorBuilder: newExecutorBuilder(tc.ctx, nil)}, + rowTypes: rightTypes, + joinKeys: innerJoinKeys, + colLens: colLens, + keyCols: tc.innerJoinKeyIdx, + compareFuncs: compareFuncs, + }, + workerWg: new(sync.WaitGroup), + isOuterJoin: false, + keyOff2IdxOff: keyOff2IdxOff, + lastColHelper: nil, + } + joiners := make([]joiner, e.ctx.GetSessionVars().IndexLookupJoinConcurrency) + for i := 0; i < e.ctx.GetSessionVars().IndexLookupJoinConcurrency; i++ { + joiners[i] = newJoiner(tc.ctx, 0, false, defaultValues, nil, leftTypes, rightTypes, nil) + } + e.joiners = joiners + return e +} + +type indexJoinType int8 + +const ( + indexInnerHashJoin indexJoinType = iota + indexOuterHashJoin + indexMergeJoin +) + +func benchmarkIndexJoinExecWithCase( + b *testing.B, + tc *indexJoinTestCase, + outerDS *mockDataSource, + innerDS *mockDataSource, + execType indexJoinType, +) { + b.ResetTimer() + for i := 0; i < b.N; i++ { + b.StopTimer() + var exec Executor + switch execType { + case indexInnerHashJoin: + exec = prepare4IndexInnerHashJoin(tc, outerDS, innerDS) + case indexOuterHashJoin: + exec = prepare4IndexOuterHashJoin(tc, outerDS, innerDS) + case indexMergeJoin: + exec = prepare4IndexMergeJoin(tc, outerDS, innerDS) + } + + tmpCtx := context.Background() + chk := newFirstChunk(exec) + outerDS.prepareChunks() + innerDS.prepareChunks() + + b.StartTimer() + if err := exec.Open(tmpCtx); err != nil { + b.Fatal(err) + } + for { + if err := exec.Next(tmpCtx, chk); err != nil { + b.Fatal(err) + } + if chk.NumRows() == 0 { + break + } + } + + if err := exec.Close(); err != nil { + b.Fatal(err) + } + b.StopTimer() + } +} + +func BenchmarkIndexJoinExec(b *testing.B) { + lvl := log.GetLevel() + log.SetLevel(zapcore.ErrorLevel) + defer log.SetLevel(lvl) + + b.ReportAllocs() + tc := defaultIndexJoinTestCase() + outerOpt := tc.getMockDataSourceOptByRows(tc.outerRows) + innerOpt := tc.getMockDataSourceOptByRows(tc.innerRows) + outerDS := buildMockDataSourceWithIndex(outerOpt, tc.innerIdx) + innerDS := buildMockDataSourceWithIndex(innerOpt, tc.innerIdx) + + tc.needOuterSort = true + b.Run(fmt.Sprintf("index merge join need outer sort %v", tc), func(b *testing.B) { + benchmarkIndexJoinExecWithCase(b, tc, outerDS, innerDS, indexMergeJoin) + }) + + tc.needOuterSort = false + b.Run(fmt.Sprintf("index merge join %v", tc), func(b *testing.B) { + benchmarkIndexJoinExecWithCase(b, tc, outerDS, innerDS, indexMergeJoin) + }) + + b.Run(fmt.Sprintf("index inner hash join %v", tc), func(b *testing.B) { + benchmarkIndexJoinExecWithCase(b, tc, outerDS, innerDS, indexInnerHashJoin) + }) + + b.Run(fmt.Sprintf("index outer hash join %v", tc), func(b *testing.B) { + benchmarkIndexJoinExecWithCase(b, tc, outerDS, innerDS, indexOuterHashJoin) + }) +} + +type mergeJoinTestCase struct { + indexJoinTestCase + childrenUsedSchema [][]bool +} + +func prepare4MergeJoin(tc *mergeJoinTestCase, leftExec, rightExec *mockDataSource) *MergeJoinExec { + outerCols, innerCols := tc.columns(), tc.columns() + + joinSchema := expression.NewSchema() + if tc.childrenUsedSchema != nil { + for i, used := range tc.childrenUsedSchema[0] { + if used { + joinSchema.Append(outerCols[i]) + } + } + for i, used := range tc.childrenUsedSchema[1] { + if used { + joinSchema.Append(innerCols[i]) + } + } + } else { + joinSchema.Append(outerCols...) + joinSchema.Append(innerCols...) + } + + outerJoinKeys := make([]*expression.Column, 0, len(tc.outerJoinKeyIdx)) + innerJoinKeys := make([]*expression.Column, 0, len(tc.innerJoinKeyIdx)) + for _, keyIdx := range tc.outerJoinKeyIdx { + outerJoinKeys = append(outerJoinKeys, outerCols[keyIdx]) + } + for _, keyIdx := range tc.innerJoinKeyIdx { + innerJoinKeys = append(innerJoinKeys, innerCols[keyIdx]) + } + compareFuncs := make([]expression.CompareFunc, 0, len(outerJoinKeys)) + outerCompareFuncs := make([]expression.CompareFunc, 0, len(outerJoinKeys)) + for i := range outerJoinKeys { + compareFuncs = append(compareFuncs, expression.GetCmpFunction(nil, outerJoinKeys[i], innerJoinKeys[i])) + outerCompareFuncs = append(outerCompareFuncs, expression.GetCmpFunction(nil, outerJoinKeys[i], outerJoinKeys[i])) + } + + defaultValues := make([]types.Datum, len(innerCols)) + + // only benchmark inner join + e := &MergeJoinExec{ + stmtCtx: tc.ctx.GetSessionVars().StmtCtx, + baseExecutor: newBaseExecutor(tc.ctx, joinSchema, stringutil.StringerStr("MergeJoin"), leftExec, rightExec), + compareFuncs: compareFuncs, + isOuterJoin: false, + } + + e.joiner = newJoiner( + tc.ctx, + 0, + false, + defaultValues, + nil, + retTypes(leftExec), + retTypes(rightExec), + tc.childrenUsedSchema, + ) + + e.innerTable = &mergeJoinTable{ + isInner: true, + childIndex: 1, + joinKeys: innerJoinKeys, + } + + e.outerTable = &mergeJoinTable{ + childIndex: 0, + filters: nil, + joinKeys: outerJoinKeys, + } + + return e +} + +func defaultMergeJoinTestCase() *mergeJoinTestCase { + return &mergeJoinTestCase{*defaultIndexJoinTestCase(), nil} +} + +func newMergeJoinBenchmark(numOuterRows, numInnerDup, numInnerRedundant int) (tc *mergeJoinTestCase, innerDS, outerDS *mockDataSource) { + ctx := mock.NewContext() + ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().SnapshotTS = 1 + ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, -1) + ctx.GetSessionVars().StmtCtx.DiskTracker = disk.NewTracker(nil, -1) + + numInnerRows := numOuterRows*numInnerDup + numInnerRedundant + itc := &indexJoinTestCase{ + outerRows: numOuterRows, + innerRows: numInnerRows, + concurrency: 4, + ctx: ctx, + outerJoinKeyIdx: []int{0, 1}, + innerJoinKeyIdx: []int{0, 1}, + innerIdx: []int{0, 1}, + rawData: wideString, + } + tc = &mergeJoinTestCase{*itc, nil} + outerOpt := mockDataSourceParameters{ + schema: expression.NewSchema(tc.columns()...), + rows: numOuterRows, + ctx: tc.ctx, + genDataFunc: func(row int, typ *types.FieldType) interface{} { + switch typ.Tp { + case mysql.TypeLong, mysql.TypeLonglong: + return int64(row) + case mysql.TypeDouble: + return float64(row) + case mysql.TypeVarString: + return tc.rawData + default: + panic("not implement") + } + }, + } + + innerOpt := mockDataSourceParameters{ + schema: expression.NewSchema(tc.columns()...), + rows: numInnerRows, + ctx: tc.ctx, + genDataFunc: func(row int, typ *types.FieldType) interface{} { + row = row / numInnerDup + switch typ.Tp { + case mysql.TypeLong, mysql.TypeLonglong: + return int64(row) + case mysql.TypeDouble: + return float64(row) + case mysql.TypeVarString: + return tc.rawData + default: + panic("not implement") + } + }, + } + + innerDS = buildMockDataSource(innerOpt) + outerDS = buildMockDataSource(outerOpt) + + return +} + +type mergeJoinType int8 + +const ( + innerMergeJoin mergeJoinType = iota +) + +func benchmarkMergeJoinExecWithCase(b *testing.B, tc *mergeJoinTestCase, innerDS, outerDS *mockDataSource, joinType mergeJoinType) { + b.ResetTimer() + for i := 0; i < b.N; i++ { + b.StopTimer() + var exec Executor + switch joinType { + case innerMergeJoin: + exec = prepare4MergeJoin(tc, innerDS, outerDS) + } + + tmpCtx := context.Background() + chk := newFirstChunk(exec) + outerDS.prepareChunks() + innerDS.prepareChunks() + + b.StartTimer() + if err := exec.Open(tmpCtx); err != nil { + b.Fatal(err) + } + for { + if err := exec.Next(tmpCtx, chk); err != nil { + b.Fatal(err) + } + if chk.NumRows() == 0 { + break + } + } + + if err := exec.Close(); err != nil { + b.Fatal(err) + } + b.StopTimer() + } +} + +func BenchmarkMergeJoinExec(b *testing.B) { + lvl := log.GetLevel() + log.SetLevel(zapcore.ErrorLevel) + defer log.SetLevel(lvl) + b.ReportAllocs() + + totalRows := 300000 + + innerDupAndRedundant := [][]int{ + {1, 0}, + {100, 0}, + {10000, 0}, + {1, 30000}, + } + + childrenUsedSchemas := [][][]bool{ + nil, + { + {true, false, false}, + {false, true, false}, + }, + } + + for _, params := range innerDupAndRedundant { + numInnerDup, numInnerRedundant := params[0], params[1] + for _, childrenUsedSchema := range childrenUsedSchemas { + tc, innerDS, outerDS := newMergeJoinBenchmark(totalRows/numInnerDup, numInnerDup, numInnerRedundant) + inlineProj := false + if childrenUsedSchema != nil { + inlineProj = true + tc.childrenUsedSchema = childrenUsedSchema + } + + b.Run(fmt.Sprintf("merge join %v InlineProj:%v", tc, inlineProj), func(b *testing.B) { + benchmarkMergeJoinExecWithCase(b, tc, outerDS, innerDS, innerMergeJoin) + }) + } + } +} + +type sortCase struct { + rows int + orderByIdx []int + ndvs []int + ctx sessionctx.Context +} + +func (tc sortCase) columns() []*expression.Column { + return []*expression.Column{ + {Index: 0, RetType: types.NewFieldType(mysql.TypeLonglong)}, + {Index: 1, RetType: types.NewFieldType(mysql.TypeLonglong)}, + } +} + +func (tc sortCase) String() string { + return fmt.Sprintf("(rows:%v, orderBy:%v, ndvs: %v)", tc.rows, tc.orderByIdx, tc.ndvs) +} + +func defaultSortTestCase() *sortCase { + ctx := mock.NewContext() + ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, -1) + tc := &sortCase{rows: 300000, orderByIdx: []int{0, 1}, ndvs: []int{0, 0}, ctx: ctx} + return tc +} + +func benchmarkSortExec(b *testing.B, cas *sortCase) { + opt := mockDataSourceParameters{ + schema: expression.NewSchema(cas.columns()...), + rows: cas.rows, + ctx: cas.ctx, + ndvs: cas.ndvs, + } + dataSource := buildMockDataSource(opt) + exec := &SortExec{ + baseExecutor: newBaseExecutor(cas.ctx, dataSource.schema, stringutil.StringerStr("sort"), dataSource), + ByItems: make([]*util.ByItems, 0, len(cas.orderByIdx)), + schema: dataSource.schema, + } + for _, idx := range cas.orderByIdx { + exec.ByItems = append(exec.ByItems, &util.ByItems{Expr: cas.columns()[idx]}) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + b.StopTimer() + tmpCtx := context.Background() + chk := newFirstChunk(exec) + dataSource.prepareChunks() + + b.StartTimer() + if err := exec.Open(tmpCtx); err != nil { + b.Fatal(err) + } + for { + if err := exec.Next(tmpCtx, chk); err != nil { + b.Fatal(err) + } + if chk.NumRows() == 0 { + break + } + } + + if err := exec.Close(); err != nil { + b.Fatal(err) + } + b.StopTimer() + } +} + +func BenchmarkSortExec(b *testing.B) { + b.ReportAllocs() + cas := defaultSortTestCase() + // all random data + cas.ndvs = []int{0, 0} + cas.orderByIdx = []int{0, 1} + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkSortExec(b, cas) + }) + + ndvs := []int{1, 10000} + for _, ndv := range ndvs { + cas.ndvs = []int{ndv, 0} + cas.orderByIdx = []int{0, 1} + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkSortExec(b, cas) + }) + + cas.ndvs = []int{ndv, 0} + cas.orderByIdx = []int{0} + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkSortExec(b, cas) + }) + + cas.ndvs = []int{ndv, 0} + cas.orderByIdx = []int{1} + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkSortExec(b, cas) + }) + } +} +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) diff --git a/executor/builder.go b/executor/builder.go index 3a0df3df8cb49..5723eb7ef3bf4 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" + plannerutil "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" @@ -816,6 +817,22 @@ func (b *executorBuilder) buildTrace(v *plannercore.Trace) Executor { builder: b, format: v.Format, } +<<<<<<< HEAD +======= + if t.format == plannercore.TraceFormatLog { + return &SortExec{ + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID(), t), + ByItems: []*plannerutil.ByItems{ + {Expr: &expression.Column{ + Index: 0, + RetType: types.NewFieldType(mysql.TypeTimestamp), + }}, + }, + schema: v.Schema(), + } + } + return t +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } // buildExplain builds a explain executor. `e.rows` collects final result to `ExplainExec`. @@ -1082,7 +1099,7 @@ func (b *executorBuilder) buildHashAgg(v *plannercore.PhysicalHashAgg) Executor e.defaultVal = chunk.NewChunkWithCapacity(retTypes(e), 1) } for _, aggDesc := range v.AggFuncs { - if aggDesc.HasDistinct { + if aggDesc.HasDistinct || len(aggDesc.OrderByItems) > 0 { e.isUnparallelExec = true } } diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index 9b1096619a79a..5e51630b3a4bc 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -23,6 +23,11 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" +<<<<<<< HEAD +======= + plannerutil "github.com/pingcap/tidb/planner/util" + "github.com/pingcap/tidb/sessionctx/variable" +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" @@ -239,3 +244,92 @@ func assertEqualStrings(c *C, got []field, expect []string) { c.Assert(string(got[i].str), Equals, expect[i]) } } +<<<<<<< HEAD +======= + +func (s *testExecSerialSuite) TestSortSpillDisk(c *C) { + originCfg := config.GetGlobalConfig() + newConf := *originCfg + newConf.OOMUseTmpStorage = true + newConf.MemQuotaQuery = 1 + config.StoreGlobalConfig(&newConf) + defer config.StoreGlobalConfig(originCfg) + + ctx := mock.NewContext() + ctx.GetSessionVars().InitChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, -1) + cas := &sortCase{rows: 2048, orderByIdx: []int{0, 1}, ndvs: []int{0, 0}, ctx: ctx} + opt := mockDataSourceParameters{ + schema: expression.NewSchema(cas.columns()...), + rows: cas.rows, + ctx: cas.ctx, + ndvs: cas.ndvs, + } + dataSource := buildMockDataSource(opt) + exec := &SortExec{ + baseExecutor: newBaseExecutor(cas.ctx, dataSource.schema, stringutil.StringerStr("sort"), dataSource), + ByItems: make([]*plannerutil.ByItems, 0, len(cas.orderByIdx)), + schema: dataSource.schema, + } + for _, idx := range cas.orderByIdx { + exec.ByItems = append(exec.ByItems, &plannerutil.ByItems{Expr: cas.columns()[idx]}) + } + tmpCtx := context.Background() + chk := newFirstChunk(exec) + dataSource.prepareChunks() + err := exec.Open(tmpCtx) + c.Assert(err, IsNil) + for { + err = exec.Next(tmpCtx, chk) + c.Assert(err, IsNil) + if chk.NumRows() == 0 { + break + } + } + // Test only 1 partition and all data in memory. + c.Assert(len(exec.partitionList), Equals, 1) + c.Assert(exec.partitionList[0].AlreadySpilled(), Equals, false) + c.Assert(exec.partitionList[0].NumRow(), Equals, 2048) + err = exec.Close() + c.Assert(err, IsNil) + + ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, 1) + dataSource.prepareChunks() + err = exec.Open(tmpCtx) + c.Assert(err, IsNil) + for { + err = exec.Next(tmpCtx, chk) + c.Assert(err, IsNil) + if chk.NumRows() == 0 { + break + } + } + // Test 2 partitions and all data in disk. + c.Assert(len(exec.partitionList), Equals, 2) + c.Assert(exec.partitionList[0].AlreadySpilled(), Equals, true) + c.Assert(exec.partitionList[1].AlreadySpilled(), Equals, true) + c.Assert(exec.partitionList[0].NumRow(), Equals, 1024) + c.Assert(exec.partitionList[1].NumRow(), Equals, 1024) + err = exec.Close() + c.Assert(err, IsNil) + + ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, 24000) + dataSource.prepareChunks() + err = exec.Open(tmpCtx) + c.Assert(err, IsNil) + for { + err = exec.Next(tmpCtx, chk) + c.Assert(err, IsNil) + if chk.NumRows() == 0 { + break + } + } + // Test only 1 partition but spill disk. + c.Assert(len(exec.partitionList), Equals, 1) + c.Assert(exec.partitionList[0].AlreadySpilled(), Equals, true) + c.Assert(exec.partitionList[0].NumRow(), Equals, 2048) + err = exec.Close() + c.Assert(err, IsNil) +} +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) diff --git a/executor/executor_required_rows_test.go b/executor/executor_required_rows_test.go index 804ccec90031d..ee30b99c2a488 100644 --- a/executor/executor_required_rows_test.go +++ b/executor/executor_required_rows_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" @@ -253,10 +254,10 @@ func (s *testExecSuite) TestSortRequiredRows(c *C) { sctx := defaultCtx() ctx := context.Background() ds := newRequiredRowsDataSource(sctx, testCase.totalRows, testCase.expectedRowsDS) - byItems := make([]*plannercore.ByItems, 0, len(testCase.groupBy)) + byItems := make([]*util.ByItems, 0, len(testCase.groupBy)) for _, groupBy := range testCase.groupBy { col := ds.Schema().Columns[groupBy] - byItems = append(byItems, &plannercore.ByItems{Expr: col}) + byItems = append(byItems, &util.ByItems{Expr: col}) } exec := buildSortExec(sctx, byItems, ds) c.Assert(exec.Open(ctx), IsNil) @@ -271,7 +272,7 @@ func (s *testExecSuite) TestSortRequiredRows(c *C) { } } -func buildSortExec(sctx sessionctx.Context, byItems []*plannercore.ByItems, src Executor) Executor { +func buildSortExec(sctx sessionctx.Context, byItems []*util.ByItems, src Executor) Executor { sortExec := SortExec{ baseExecutor: newBaseExecutor(sctx, src.Schema(), nil, src), ByItems: byItems, @@ -360,10 +361,10 @@ func (s *testExecSuite) TestTopNRequiredRows(c *C) { sctx := defaultCtx() ctx := context.Background() ds := newRequiredRowsDataSource(sctx, testCase.totalRows, testCase.expectedRowsDS) - byItems := make([]*plannercore.ByItems, 0, len(testCase.groupBy)) + byItems := make([]*util.ByItems, 0, len(testCase.groupBy)) for _, groupBy := range testCase.groupBy { col := ds.Schema().Columns[groupBy] - byItems = append(byItems, &plannercore.ByItems{Expr: col}) + byItems = append(byItems, &util.ByItems{Expr: col}) } exec := buildTopNExec(sctx, testCase.topNOffset, testCase.topNCount, byItems, ds) c.Assert(exec.Open(ctx), IsNil) @@ -378,7 +379,7 @@ func (s *testExecSuite) TestTopNRequiredRows(c *C) { } } -func buildTopNExec(ctx sessionctx.Context, offset, count int, byItems []*plannercore.ByItems, src Executor) Executor { +func buildTopNExec(ctx sessionctx.Context, offset, count int, byItems []*util.ByItems, src Executor) Executor { sortExec := SortExec{ baseExecutor: newBaseExecutor(ctx, src.Schema(), nil, src), ByItems: byItems, diff --git a/executor/sort.go b/executor/sort.go index fefac4cda4c1a..3867de92b7e2e 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -23,6 +23,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/tidb/expression" plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/memory" @@ -35,7 +36,7 @@ var rowChunksLabel fmt.Stringer = stringutil.StringerStr("rowChunks") type SortExec struct { baseExecutor - ByItems []*plannercore.ByItems + ByItems []*util.ByItems Idx int fetched bool schema *expression.Schema diff --git a/expression/aggregation/agg_to_pb.go b/expression/aggregation/agg_to_pb.go index 59d09db237701..a6a003a01a86f 100644 --- a/expression/aggregation/agg_to_pb.go +++ b/expression/aggregation/agg_to_pb.go @@ -26,6 +26,9 @@ func AggFuncToPBExpr(sc *stmtctx.StatementContext, client kv.Client, aggFunc *Ag if aggFunc.HasDistinct { return nil } + if len(aggFunc.OrderByItems) > 0 { + return nil + } pc := expression.NewPBConverter(client, sc) var tp tipb.ExprType switch aggFunc.Name { diff --git a/expression/aggregation/aggregation.go b/expression/aggregation/aggregation.go index ba98bb37bbe0c..676f93f810e5f 100644 --- a/expression/aggregation/aggregation.go +++ b/expression/aggregation/aggregation.go @@ -250,3 +250,31 @@ func IsAllFirstRow(aggFuncs []*AggFuncDesc) bool { } return true } +<<<<<<< HEAD +======= + +// CheckAggPushDown checks whether an agg function can be pushed to storage. +func CheckAggPushDown(aggFunc *AggFuncDesc, storeType kv.StoreType) bool { + if len(aggFunc.OrderByItems) > 0 { + return false + } + ret := true + switch storeType { + case kv.TiFlash: + ret = CheckAggPushFlash(aggFunc) + } + if ret { + ret = expression.IsPushDownEnabled(strings.ToLower(aggFunc.Name), storeType) + } + return ret +} + +// CheckAggPushFlash checks whether an agg function can be pushed to flash storage. +func CheckAggPushFlash(aggFunc *AggFuncDesc) bool { + switch aggFunc.Name { + case ast.AggFuncSum, ast.AggFuncCount, ast.AggFuncMin, ast.AggFuncMax, ast.AggFuncAvg, ast.AggFuncFirstRow: + return true + } + return false +} +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) diff --git a/expression/aggregation/descriptor.go b/expression/aggregation/descriptor.go index 66f5f3346c805..24a7e7f5fc8d8 100644 --- a/expression/aggregation/descriptor.go +++ b/expression/aggregation/descriptor.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" @@ -34,6 +35,8 @@ type AggFuncDesc struct { Mode AggFunctionMode // HasDistinct represents whether the aggregation function contains distinct attribute. HasDistinct bool + // OrderByItems represents the order by clause used in GROUP_CONCAT + OrderByItems []*util.ByItems } // NewAggFuncDesc creates an aggregation function signature descriptor. @@ -50,6 +53,14 @@ func (a *AggFuncDesc) Equal(ctx sessionctx.Context, other *AggFuncDesc) bool { if a.HasDistinct != other.HasDistinct { return false } + if len(a.OrderByItems) != len(other.OrderByItems) { + return false + } + for i := range a.OrderByItems { + if !a.OrderByItems[i].Equal(ctx, other.OrderByItems[i]) { + return false + } + } return a.baseFuncDesc.equal(ctx, &other.baseFuncDesc) } @@ -57,6 +68,10 @@ func (a *AggFuncDesc) Equal(ctx sessionctx.Context, other *AggFuncDesc) bool { func (a *AggFuncDesc) Clone() *AggFuncDesc { clone := *a clone.baseFuncDesc = *a.baseFuncDesc.clone() + clone.OrderByItems = make([]*util.ByItems, len(a.OrderByItems)) + for i, byItem := range a.OrderByItems { + clone.OrderByItems[i] = byItem.Clone() + } return &clone } diff --git a/expression/aggregation/explain.go b/expression/aggregation/explain.go index 0a6a01a4ed8b7..b001a21c23d1e 100644 --- a/expression/aggregation/explain.go +++ b/expression/aggregation/explain.go @@ -16,6 +16,8 @@ package aggregation import ( "bytes" "fmt" + + "github.com/pingcap/parser/ast" ) // ExplainAggFunc generates explain information for a aggregation function. @@ -26,10 +28,25 @@ func ExplainAggFunc(agg *AggFuncDesc) string { buffer.WriteString("distinct ") } for i, arg := range agg.Args { - buffer.WriteString(arg.ExplainInfo()) - if i+1 < len(agg.Args) { + if agg.Name == ast.AggFuncGroupConcat && i == len(agg.Args)-1 { + if len(agg.OrderByItems) > 0 { + buffer.WriteString(" order by ") + for i, item := range agg.OrderByItems { + order := "asc" + if item.Desc { + order = "desc" + } + fmt.Fprintf(&buffer, "%s %s", item.Expr.ExplainInfo(), order) + if i+1 < len(agg.OrderByItems) { + buffer.WriteString(", ") + } + } + } + buffer.WriteString(" separator ") + } else if i != 0 { buffer.WriteString(", ") } + buffer.WriteString(arg.ExplainInfo()) } buffer.WriteString(")") return buffer.String() diff --git a/planner/cascades/enforcer_rules.go b/planner/cascades/enforcer_rules.go index e3d41b0a88719..a526bad3fe577 100644 --- a/planner/cascades/enforcer_rules.go +++ b/planner/cascades/enforcer_rules.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/tidb/planner/implementation" "github.com/pingcap/tidb/planner/memo" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" ) // Enforcer defines the interface for enforcer rules. @@ -55,11 +56,18 @@ func (e *OrderEnforcer) NewProperty(prop *property.PhysicalProperty) (newProp *p // OnEnforce adds sort operator to satisfy required order property. func (e *OrderEnforcer) OnEnforce(reqProp *property.PhysicalProperty, child memo.Implementation) (impl memo.Implementation) { +<<<<<<< HEAD sort := &plannercore.PhysicalSort{ ByItems: make([]*plannercore.ByItems, 0, len(reqProp.Items)), } +======= + childPlan := child.GetPlan() + sort := plannercore.PhysicalSort{ + ByItems: make([]*util.ByItems, 0, len(reqProp.Items)), + }.Init(childPlan.SCtx(), childPlan.Stats(), childPlan.SelectBlockOffset(), &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64}) +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) for _, item := range reqProp.Items { - item := &plannercore.ByItems{ + item := &util.ByItems{ Expr: item.Col, Desc: item.Desc, } diff --git a/planner/cascades/testdata/transformation_rules_suite_out.json b/planner/cascades/testdata/transformation_rules_suite_out.json new file mode 100644 index 0000000000000..ba8bbcae8e221 --- /dev/null +++ b/planner/cascades/testdata/transformation_rules_suite_out.json @@ -0,0 +1,2356 @@ +[ + { + "Name": "TestPredicatePushDown", + "Cases": [ + { + "SQL": "select a, b from (select a, b from t as t1 order by a) as t2 where t2.b > 10", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b]", + " Projection_5 input:[Group#1], test.t.a, test.t.b", + "Group#1 Schema:[test.t.a,test.t.b]", + " Sort_3 input:[Group#2], test.t.a:asc", + "Group#2 Schema:[test.t.a,test.t.b]", + " Projection_2 input:[Group#3], test.t.a, test.t.b", + "Group#3 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_8 input:[Group#4], table:t1", + "Group#4 Schema:[test.t.a,test.t.b]", + " Selection_9 input:[Group#5], gt(test.t.b, 10)", + "Group#5 Schema:[test.t.a,test.t.b]", + " TableScan_7 table:t1, pk col:test.t.a" + ] + }, + { + "SQL": "select a, b from (select a, b from t as t1 order by a) as t2 where t2.a > 10", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b]", + " Projection_5 input:[Group#1], test.t.a, test.t.b", + "Group#1 Schema:[test.t.a,test.t.b]", + " Sort_3 input:[Group#2], test.t.a:asc", + "Group#2 Schema:[test.t.a,test.t.b]", + " Projection_2 input:[Group#3], test.t.a, test.t.b", + "Group#3 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_8 input:[Group#4], table:t1", + "Group#4 Schema:[test.t.a,test.t.b]", + " TableScan_10 table:t1, pk col:test.t.a, cond:[gt(test.t.a, 10)]" + ] + }, + { + "SQL": "select a, b from (select a, b, a+b as a_b from t as t1) as t2 where a_b > 10 and b = 1", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b]", + " Projection_4 input:[Group#1], test.t.a, test.t.b", + "Group#1 Schema:[test.t.a,test.t.b,Column#13]", + " Projection_2 input:[Group#2], test.t.a, test.t.b, plus(test.t.a, test.t.b)->Column#13", + "Group#2 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_7 input:[Group#3], table:t1", + "Group#3 Schema:[test.t.a,test.t.b]", + " Selection_8 input:[Group#4], eq(test.t.b, 1), gt(plus(test.t.a, test.t.b), 10)", + "Group#4 Schema:[test.t.a,test.t.b]", + " TableScan_6 table:t1, pk col:test.t.a" + ] + }, + { + "SQL": "select b, @i:=@i+1 as ii from (select b, @i:=0 from t as t1) as t2 where @i < 10", + "Result": [ + "Group#0 Schema:[test.t.b,Column#14]", + " Projection_4 input:[Group#1], test.t.b, setvar(i, cast(plus(cast(getvar(i), double BINARY), 1), var_string(5)))->Column#14", + "Group#1 Schema:[test.t.b,Column#13]", + " Selection_3 input:[Group#2], lt(cast(getvar(\"i\")), 10)", + "Group#2 Schema:[test.t.b,Column#13]", + " Projection_2 input:[Group#3], test.t.b, setvar(i, 0)->Column#13", + "Group#3 Schema:[test.t.b]", + " TiKVSingleGather_6 input:[Group#4], table:t1", + "Group#4 Schema:[test.t.b]", + " TableScan_5 table:t1" + ] + }, + { + "SQL": "select b, @i:=@i+1 as ii from (select a, b, @i:=0 from t as t1) as t2 where @i < 10 and a > 10", + "Result": [ + "Group#0 Schema:[test.t.b,Column#14]", + " Projection_4 input:[Group#1], test.t.b, setvar(i, cast(plus(cast(getvar(i), double BINARY), 1), var_string(5)))->Column#14", + "Group#1 Schema:[test.t.a,test.t.b,Column#13]", + " Selection_6 input:[Group#2], lt(cast(getvar(\"i\")), 10)", + "Group#2 Schema:[test.t.a,test.t.b,Column#13]", + " Projection_2 input:[Group#3], test.t.a, test.t.b, setvar(i, 0)->Column#13", + "Group#3 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_8 input:[Group#4], table:t1", + "Group#4 Schema:[test.t.a,test.t.b]", + " TableScan_10 table:t1, pk col:test.t.a, cond:[gt(test.t.a, 10)]" + ] + }, + { + "SQL": "select a, max(b) from t group by a having a > 1", + "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], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", + "Group#2 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_8 input:[Group#3], table:t", + "Group#3 Schema:[test.t.a,test.t.b]", + " TableScan_10 table:t, pk col:test.t.a, cond:[gt(test.t.a, 1)]" + ] + }, + { + "SQL": "select a, avg(b) from t group by a having a > 1 and max(b) > 10", + "Result": [ + "Group#0 Schema:[test.t.a,Column#16]", + " Projection_5 input:[Group#1], test.t.a, Column#13", + "Group#1 Schema:[test.t.a,Column#13,Column#14]", + " Projection_3 input:[Group#2], test.t.a, Column#13, Column#14", + "Group#2 Schema:[Column#13,Column#14,test.t.a]", + " Selection_8 input:[Group#3], gt(Column#14, 10)", + "Group#3 Schema:[Column#13,Column#14,test.t.a]", + " Aggregation_2 input:[Group#4], group by:test.t.a, funcs:avg(test.t.b), max(test.t.b), firstrow(test.t.a)", + "Group#4 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_10 input:[Group#5], table:t", + "Group#5 Schema:[test.t.a,test.t.b]", + " TableScan_12 table:t, pk col:test.t.a, cond:[gt(test.t.a, 1)]" + ] + }, + { + "SQL": "select t1.a, t1.b, t2.b from t t1, t t2 where t1.a = t2.a and t2.b = t1.b and t1.a > 10 and t2.b > 10 and t1.a > t2.b", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.b]", + " Projection_5 input:[Group#1], test.t.a, test.t.b, test.t.b", + "Group#1 Schema:[test.t.a,test.t.b,test.t.a,test.t.b]", + " Join_3 input:[Group#2,Group#3], inner join, equal:[eq(test.t.a, test.t.a) eq(test.t.b, test.t.b)], other cond:gt(test.t.a, test.t.b)", + "Group#2 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_9 input:[Group#4], table:t1", + "Group#4 Schema:[test.t.a,test.t.b]", + " Selection_12 input:[Group#5], gt(test.t.a, test.t.b), gt(test.t.b, 10)", + "Group#5 Schema:[test.t.a,test.t.b]", + " TableScan_11 table:t1, pk col:test.t.a, cond:[gt(test.t.a, 10)]", + "Group#3 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_14 input:[Group#6], table:t2", + "Group#6 Schema:[test.t.a,test.t.b]", + " Selection_17 input:[Group#7], gt(test.t.a, test.t.b), gt(test.t.b, 10)", + "Group#7 Schema:[test.t.a,test.t.b]", + " TableScan_16 table:t2, pk col:test.t.a, cond:[gt(test.t.a, 10)]" + ] + }, + { + "SQL": "select t1.a, t1.b from t t1, t t2 where t1.a = t2.a and t1.a = 10 and t2.a = 5", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b]", + " Projection_5 input:[Group#1], test.t.a, test.t.b", + "Group#1 Schema:[test.t.a,test.t.b,test.t.a]", + " TableDual_6 rowcount:0" + ] + }, + { + "SQL": "select a, f from t where f > 1", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.f]", + " Projection_3 input:[Group#1], test.t.a, test.t.f", + "Group#1 Schema:[test.t.a,test.t.f]", + " TiKVSingleGather_5 input:[Group#2], table:t", + " TiKVSingleGather_7 input:[Group#3], table:t, index:f", + " TiKVSingleGather_9 input:[Group#4], table:t, index:f_g", + "Group#2 Schema:[test.t.a,test.t.f]", + " Selection_10 input:[Group#5], gt(test.t.f, 1)", + "Group#5 Schema:[test.t.a,test.t.f]", + " TableScan_4 table:t, pk col:test.t.a", + "Group#3 Schema:[test.t.a,test.t.f]", + " IndexScan_13 table:t, index:f, cond:[gt(test.t.f, 1)]", + "Group#4 Schema:[test.t.a,test.t.f]", + " IndexScan_14 table:t, index:f, g, cond:[gt(test.t.f, 1)]" + ] + }, + { + "SQL": "select a, f from (select a, f, g from t where f = 1) t1 where g > 1", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.f]", + " Projection_5 input:[Group#1], test.t.a, test.t.f", + "Group#1 Schema:[test.t.a,test.t.f,test.t.g]", + " Projection_3 input:[Group#2], test.t.a, test.t.f, test.t.g", + "Group#2 Schema:[test.t.a,test.t.f,test.t.g]", + " TiKVSingleGather_9 input:[Group#3], table:t", + " TiKVSingleGather_11 input:[Group#4], table:t, index:f_g", + "Group#3 Schema:[test.t.a,test.t.f,test.t.g]", + " Selection_12 input:[Group#5], eq(test.t.f, 1), gt(test.t.g, 1)", + "Group#5 Schema:[test.t.a,test.t.f,test.t.g]", + " TableScan_8 table:t, pk col:test.t.a", + "Group#4 Schema:[test.t.a,test.t.f,test.t.g]", + " IndexScan_14 table:t, index:f, g, cond:[eq(test.t.f, 1) gt(test.t.g, 1)]" + ] + }, + { + "SQL": "select a, f from t where g > 1 and f > 1", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.f]", + " Projection_3 input:[Group#1], test.t.a, test.t.f", + "Group#1 Schema:[test.t.a,test.t.f,test.t.g]", + " TiKVSingleGather_5 input:[Group#2], table:t", + " TiKVSingleGather_7 input:[Group#3], table:t, index:f_g", + "Group#2 Schema:[test.t.a,test.t.f,test.t.g]", + " Selection_8 input:[Group#4], gt(test.t.f, 1), gt(test.t.g, 1)", + "Group#4 Schema:[test.t.a,test.t.f,test.t.g]", + " TableScan_4 table:t, pk col:test.t.a", + "Group#3 Schema:[test.t.a,test.t.f,test.t.g]", + " Selection_11 input:[Group#5], gt(test.t.g, 1)", + "Group#5 Schema:[test.t.a,test.t.f,test.t.g]", + " IndexScan_10 table:t, index:f, g, cond:[gt(test.t.f, 1)]" + ] + }, + { + "SQL": "select t1.a, t1.b from t t1, t t2 where t1.a = t2.a and t1.a = 10 and t2.a = 5", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b]", + " Projection_5 input:[Group#1], test.t.a, test.t.b", + "Group#1 Schema:[test.t.a,test.t.b,test.t.a]", + " TableDual_6 rowcount:0" + ] + }, + { + "SQL": "select a, b from ((select a, b from t) union all(select c as a, d as b from t)) as t1 where a > 1", + "Result": [ + "Group#0 Schema:[Column#25,Column#26]", + " Projection_9 input:[Group#1], Column#25, Column#26", + "Group#1 Schema:[Column#25,Column#26]", + " Union_5 input:[Group#2,Group#3]", + "Group#2 Schema:[Column#25,Column#26]", + " Projection_6 input:[Group#4], test.t.a, test.t.b", + "Group#4 Schema:[test.t.a,test.t.b]", + " Projection_4 input:[Group#5], test.t.a, test.t.b", + "Group#5 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_15 input:[Group#6], table:t", + "Group#6 Schema:[test.t.a,test.t.b]", + " TableScan_17 table:t, pk col:test.t.a, cond:[gt(test.t.a, 1)]", + "Group#3 Schema:[Column#25,Column#26]", + " Projection_7 input:[Group#7], test.t.c, test.t.d", + "Group#7 Schema:[test.t.c,test.t.d]", + " Projection_2 input:[Group#8], test.t.c, test.t.d", + "Group#8 Schema:[test.t.c,test.t.d]", + " TiKVSingleGather_19 input:[Group#9], table:t", + " TiKVSingleGather_21 input:[Group#10], table:t, index:c_d_e", + "Group#9 Schema:[test.t.c,test.t.d]", + " Selection_22 input:[Group#11], gt(test.t.c, 1)", + "Group#11 Schema:[test.t.c,test.t.d]", + " TableScan_18 table:t", + "Group#10 Schema:[test.t.c,test.t.d]", + " IndexScan_24 table:t, index:c, d, e, cond:[gt(test.t.c, 1)]" + ] + }, + { + "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", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b]", + " Projection_7 input:[Group#1], test.t.a, test.t.b", + "Group#1 Schema:[test.t.a,test.t.b,Column#14]", + " Projection_5 input:[Group#2], test.t.a, test.t.b, Column#14", + "Group#2 Schema:[test.t.a,test.t.b,Column#14]", + " Selection_10 input:[Group#3], eq(test.t.b, Column#14), lt(test.t.a, 10)", + "Group#3 Schema:[test.t.a,test.t.b,Column#14]", + " Window_4 input:[Group#4]", + "Group#4 Schema:[test.t.a,test.t.b]", + " Projection_3 input:[Group#5], test.t.a, test.t.b", + "Group#5 Schema:[test.t.a,test.t.b]", + " Projection_2 input:[Group#6], test.t.a, test.t.b", + "Group#6 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_14 input:[Group#7], table:t", + "Group#7 Schema:[test.t.a,test.t.b]", + " Selection_15 input:[Group#8], gt(test.t.b, 10)", + "Group#8 Schema:[test.t.a,test.t.b]", + " TableScan_13 table:t, pk col:test.t.a" + ] + }, + { + "SQL": "select b, c from (select b, c from t where b > 1 and c > 1) as t1 where b > 2 and c > 2", + "Result": [ + "Group#0 Schema:[test.t.b,test.t.c]", + " Projection_5 input:[Group#1], test.t.b, test.t.c", + "Group#1 Schema:[test.t.b,test.t.c]", + " Projection_3 input:[Group#2], test.t.b, test.t.c", + "Group#2 Schema:[test.t.b,test.t.c]", + " TiKVSingleGather_9 input:[Group#3], table:t", + "Group#3 Schema:[test.t.b,test.t.c]", + " Selection_10 input:[Group#4], gt(test.t.b, 1), gt(test.t.b, 2), gt(test.t.c, 1), gt(test.t.c, 2)", + "Group#4 Schema:[test.t.b,test.t.c]", + " TableScan_8 table:t" + ] + } + ] + }, + { + "Name": "TestAggPushDownGather", + "Cases": [ + { + "SQL": "select b, sum(a) from t group by b", + "Result": [ + "Group#0 Schema:[test.t.b,Column#13], UniqueKey:[test.t.b]", + " Projection_3 input:[Group#1], test.t.b, Column#13", + "Group#1 Schema:[Column#13,test.t.b], UniqueKey:[test.t.b]", + " Aggregation_2 input:[Group#2], group by:test.t.b, funcs:sum(test.t.a), firstrow(test.t.b)", + " Aggregation_7 input:[Group#3], group by:test.t.b, funcs:sum(Column#14), firstrow(test.t.b)", + "Group#2 Schema:[test.t.a,test.t.b], UniqueKey:[test.t.a]", + " TiKVSingleGather_5 input:[Group#4], table:t", + "Group#4 Schema:[test.t.a,test.t.b], UniqueKey:[test.t.a]", + " TableScan_4 table:t, pk col:test.t.a", + "Group#3 Schema:[Column#14,test.t.b]", + " TiKVSingleGather_5 input:[Group#5], table:t", + "Group#5 Schema:[Column#14,test.t.b]", + " Aggregation_6 input:[Group#4], group by:test.t.b, funcs:sum(test.t.a)" + ] + }, + { + "SQL": "select b, sum(a) from t group by c, b", + "Result": [ + "Group#0 Schema:[test.t.b,Column#13]", + " Projection_3 input:[Group#1], test.t.b, Column#13", + "Group#1 Schema:[Column#13,test.t.b]", + " Aggregation_2 input:[Group#2], group by:test.t.b, test.t.c, funcs:sum(test.t.a), firstrow(test.t.b)", + " Aggregation_7 input:[Group#3], group by:test.t.b, test.t.c, funcs:sum(Column#14), firstrow(test.t.b)", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c], UniqueKey:[test.t.a]", + " TiKVSingleGather_5 input:[Group#4], table:t", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c], UniqueKey:[test.t.a]", + " TableScan_4 table:t, pk col:test.t.a", + "Group#3 Schema:[Column#14,test.t.c,test.t.b]", + " TiKVSingleGather_5 input:[Group#5], table:t", + "Group#5 Schema:[Column#14,test.t.c,test.t.b]", + " Aggregation_6 input:[Group#4], group by:test.t.b, test.t.c, funcs:sum(test.t.a)" + ] + }, + { + "SQL": "select b, sum(a) from t group by sin(b)+sin(c), b", + "Result": [ + "Group#0 Schema:[test.t.b,Column#13]", + " Projection_3 input:[Group#1], test.t.b, Column#13", + "Group#1 Schema:[Column#13,test.t.b]", + " Aggregation_2 input:[Group#2], group by:plus(sin(cast(test.t.b)), sin(cast(test.t.c))), test.t.b, funcs:sum(test.t.a), firstrow(test.t.b)", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c], UniqueKey:[test.t.a]", + " TiKVSingleGather_5 input:[Group#3], table:t", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c], UniqueKey:[test.t.a]", + " TableScan_4 table:t, pk col:test.t.a" + ] + } + ] + }, + { + "Name": "TestTopNRules", + "Cases": [ + { + "SQL": "select b from t order by a limit 2", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_5 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b,test.t.a]", + " Projection_2 input:[Group#2], test.t.b, test.t.a", + "Group#2 Schema:[test.t.a,test.t.b]", + " TopN_7 input:[Group#3], test.t.a:asc, offset:0, count:2", + "Group#3 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_9 input:[Group#4], table:t", + "Group#4 Schema:[test.t.a,test.t.b]", + " TopN_10 input:[Group#5], test.t.a:asc, offset:0, count:2", + "Group#5 Schema:[test.t.a,test.t.b]", + " TableScan_8 table:t, pk col:test.t.a" + ] + }, + { + "SQL": "select b from t limit 2", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_2 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b]", + " Limit_3 input:[Group#2], offset:0, count:2", + "Group#2 Schema:[test.t.b]", + " TiKVSingleGather_5 input:[Group#3], table:t", + "Group#3 Schema:[test.t.b]", + " Limit_6 input:[Group#4], offset:0, count:2", + "Group#4 Schema:[test.t.b]", + " TableScan_4 table:t" + ] + }, + { + "SQL": "select a+b from t order by a limit 1 offset 2", + "Result": [ + "Group#0 Schema:[Column#14]", + " Projection_5 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13,test.t.a]", + " Projection_2 input:[Group#2], plus(test.t.a, test.t.b)->Column#13, test.t.a", + "Group#2 Schema:[test.t.a,test.t.b]", + " TopN_7 input:[Group#3], test.t.a:asc, offset:2, count:1", + "Group#3 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_9 input:[Group#4], table:t", + "Group#4 Schema:[test.t.a,test.t.b]", + " TopN_10 input:[Group#5], test.t.a:asc, offset:0, count:3", + "Group#5 Schema:[test.t.a,test.t.b]", + " TableScan_8 table:t, pk col:test.t.a" + ] + }, + { + "SQL": "select c from t order by t.a limit 1", + "Result": [ + "Group#0 Schema:[test.t.c]", + " Projection_5 input:[Group#1], test.t.c", + "Group#1 Schema:[test.t.c,test.t.a]", + " Projection_2 input:[Group#2], test.t.c, test.t.a", + "Group#2 Schema:[test.t.a,test.t.c]", + " TopN_7 input:[Group#3], test.t.a:asc, offset:0, count:1", + " TopN_7 input:[Group#4], test.t.a:asc, offset:0, count:1", + "Group#3 Schema:[test.t.a,test.t.c]", + " TiKVSingleGather_11 input:[Group#5], table:t, index:c_d_e", + "Group#5 Schema:[test.t.a,test.t.c]", + " TopN_13 input:[Group#6], test.t.a:asc, offset:0, count:1", + "Group#6 Schema:[test.t.a,test.t.c]", + " IndexScan_10 table:t, index:c, d, e", + "Group#4 Schema:[test.t.a,test.t.c]", + " TiKVSingleGather_9 input:[Group#7], table:t", + "Group#7 Schema:[test.t.a,test.t.c]", + " TopN_12 input:[Group#8], test.t.a:asc, offset:0, count:1", + "Group#8 Schema:[test.t.a,test.t.c]", + " TableScan_8 table:t, pk col:test.t.a" + ] + }, + { + "SQL": "select c from t order by t.a + t.b limit 1", + "Result": [ + "Group#0 Schema:[test.t.c]", + " Projection_5 input:[Group#1], test.t.c", + "Group#1 Schema:[test.t.c,test.t.a,test.t.b]", + " Projection_2 input:[Group#2], test.t.c, test.t.a, test.t.b", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", + " TopN_7 input:[Group#3], plus(test.t.a, test.t.b):asc, offset:0, count:1", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c]", + " TiKVSingleGather_9 input:[Group#4], table:t", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c]", + " TopN_10 input:[Group#5], plus(test.t.a, test.t.b):asc, offset:0, count:1", + "Group#5 Schema:[test.t.a,test.t.b,test.t.c]", + " TableScan_8 table:t, pk col:test.t.a" + ] + }, + { + "SQL": "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)", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c]", + " Projection_9 input:[Group#1], test.t.a, test.t.b, test.t.c", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c]", + " Apply_8 input:[Group#2,Group#3], semi join, equal:[eq(test.t.a, test.t.a)]", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", + " TiKVSingleGather_13 input:[Group#4], table:t1", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c]", + " TableScan_12 table:t1, pk col:test.t.a", + "Group#3 Schema:[test.t.a]", + " Projection_5 input:[Group#5], test.t.a", + "Group#5 Schema:[test.t.a,test.t.b]", + " TopN_11 input:[Group#6], , offset:0, count:1", + "Group#6 Schema:[test.t.a,test.t.b]", + " Selection_4 input:[Group#7], gt(test.t.b, test.t.b)", + "Group#7 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_15 input:[Group#8], table:t2", + "Group#8 Schema:[test.t.a,test.t.b]", + " TableScan_14 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "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)", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c]", + " Projection_11 input:[Group#1], test.t.a, test.t.b, test.t.c", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c]", + " Apply_10 input:[Group#2,Group#3], semi join, equal:[eq(test.t.a, test.t.a)]", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", + " TiKVSingleGather_16 input:[Group#4], table:t1", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c]", + " TableScan_15 table:t1, pk col:test.t.a", + "Group#3 Schema:[test.t.a]", + " Projection_9 input:[Group#5], test.t.a", + "Group#5 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#6], test.t.a, Column#25", + "Group#6 Schema:[test.t.a,Column#25]", + " Projection_5 input:[Group#7], test.t.a, test.t.b", + "Group#7 Schema:[test.t.a,test.t.b]", + " TopN_14 input:[Group#8], test.t.b:asc, offset:0, count:1", + "Group#8 Schema:[test.t.a,test.t.b]", + " Selection_4 input:[Group#9], gt(test.t.b, test.t.b)", + "Group#9 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_18 input:[Group#10], table:t2", + "Group#10 Schema:[test.t.a,test.t.b]", + " TableScan_17 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "select a, b from (select @i as a, @i := @i+1 as b from t) t order by a desc limit 1", + "Result": [ + "Group#0 Schema:[Column#13,Column#14]", + " Projection_3 input:[Group#1], Column#13, Column#14", + "Group#1 Schema:[Column#13,Column#14]", + " TopN_7 input:[Group#2], Column#13:desc, offset:0, count:1", + "Group#2 Schema:[Column#13,Column#14]", + " Projection_2 input:[Group#3], getvar(i)->Column#13, setvar(i, cast(plus(cast(getvar(i), double BINARY), 1), var_string(5)))->Column#14", + "Group#3 Schema:[test.t.a]", + " TiKVSingleGather_9 input:[Group#4], table:t", + " TiKVSingleGather_21 input:[Group#5], table:t, index:e_d_c_str_prefix", + " TiKVSingleGather_19 input:[Group#6], table:t, index:c_d_e_str", + " TiKVSingleGather_17 input:[Group#7], table:t, index:f_g", + " TiKVSingleGather_15 input:[Group#8], table:t, index:g", + " TiKVSingleGather_13 input:[Group#9], table:t, index:f", + " TiKVSingleGather_11 input:[Group#10], table:t, index:c_d_e", + "Group#4 Schema:[test.t.a]", + " TableScan_8 table:t, pk col:test.t.a", + "Group#5 Schema:[test.t.a]", + " IndexScan_20 table:t, index:e_str, d_str, c_str", + "Group#6 Schema:[test.t.a]", + " IndexScan_18 table:t, index:c_str, d_str, e_str", + "Group#7 Schema:[test.t.a]", + " IndexScan_16 table:t, index:f, g", + "Group#8 Schema:[test.t.a]", + " IndexScan_14 table:t, index:g", + "Group#9 Schema:[test.t.a]", + " IndexScan_12 table:t, index:f", + "Group#10 Schema:[test.t.a]", + " IndexScan_10 table:t, index:c, d, e" + ] + }, + { + "SQL": "select * from t t1 left join t t2 on t1.b = t2.b order by t1.b limit 1", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TopN_8 input:[Group#2], test.t.b:asc, offset:0, count:1", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Join_3 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.b, test.t.b)]", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TopN_9 input:[Group#5], test.t.b:asc, offset:0, count:1", + "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_11 input:[Group#6], table:t1", + "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TopN_12 input:[Group#7], test.t.b:asc, offset:0, count:1", + "Group#7 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_10 table:t1, pk col:test.t.a", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_14 input:[Group#8], table:t2", + "Group#8 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_13 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "select * from t t1 left join t t2 on t1.b = t2.b order by t1.a, t1.c limit 1", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TopN_8 input:[Group#2], test.t.a:asc, test.t.c:asc, offset:0, count:1", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Join_3 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.b, test.t.b)]", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TopN_9 input:[Group#5], test.t.a:asc, test.t.c:asc, offset:0, count:1", + "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_11 input:[Group#6], table:t1", + "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TopN_12 input:[Group#7], test.t.a:asc, test.t.c:asc, offset:0, count:1", + "Group#7 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_10 table:t1, pk col:test.t.a", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_14 input:[Group#8], table:t2", + "Group#8 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_13 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "select * from t t1 left join t t2 on t1.b = t2.b order by t2.a, t2.c limit 1", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TopN_8 input:[Group#2], test.t.a:asc, test.t.c:asc, offset:0, count:1", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Join_3 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.b, test.t.b)]", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_10 input:[Group#5], table:t1", + "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_9 table:t1, pk col:test.t.a", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_12 input:[Group#6], table:t2", + "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_11 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "select * from t t1 left join t t2 on t1.b = t2.b order by t1.a, t2.c limit 1", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TopN_8 input:[Group#2], test.t.a:asc, test.t.c:asc, offset:0, count:1", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Join_3 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.b, test.t.b)]", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_10 input:[Group#5], table:t1", + "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_9 table:t1, pk col:test.t.a", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_12 input:[Group#6], table:t2", + "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_11 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "select * from t t1 right join t t2 on t1.b = t2.b order by t1.a, t1.c limit 1", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TopN_8 input:[Group#2], test.t.a:asc, test.t.c:asc, offset:0, count:1", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Join_3 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.b, test.t.b)]", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_10 input:[Group#5], table:t1", + "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_9 table:t1, pk col:test.t.a", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_12 input:[Group#6], table:t2", + "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_11 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "select * from t t1 right join t t2 on t1.b = t2.b order by t2.a, t2.c limit 1", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TopN_8 input:[Group#2], test.t.a:asc, test.t.c:asc, offset:0, count:1", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Join_3 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.b, test.t.b)]", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_11 input:[Group#5], table:t1", + "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_10 table:t1, pk col:test.t.a", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TopN_9 input:[Group#6], test.t.a:asc, test.t.c:asc, offset:0, count:1", + "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_13 input:[Group#7], table:t2", + "Group#7 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TopN_14 input:[Group#8], test.t.a:asc, test.t.c:asc, offset:0, count:1", + "Group#8 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_12 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "select * from t t1 right join t t2 on t1.b = t2.b order by t1.a, t2.c limit 1", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TopN_8 input:[Group#2], test.t.a:asc, test.t.c:asc, offset:0, count:1", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Join_3 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.b, test.t.b)]", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_10 input:[Group#5], table:t1", + "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_9 table:t1, pk col:test.t.a", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_12 input:[Group#6], table:t2", + "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_11 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "select * from t t1 left join t t2 on t1.b = t2.b limit 1", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_5 input:[Group#2], offset:0, count:1", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Join_3 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.b, test.t.b)]", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_6 input:[Group#5], offset:0, count:1", + "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_8 input:[Group#6], table:t1", + "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_9 input:[Group#7], offset:0, count:1", + "Group#7 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_7 table:t1, pk col:test.t.a", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_11 input:[Group#8], table:t2", + "Group#8 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_10 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "select * from t t1 left join t t2 on t1.b = t2.b limit 5 offset 4", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_5 input:[Group#2], offset:4, count:5", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Join_3 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.b, test.t.b)]", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_6 input:[Group#5], offset:0, count:9", + "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_8 input:[Group#6], table:t1", + "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_9 input:[Group#7], offset:0, count:9", + "Group#7 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_7 table:t1, pk col:test.t.a", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_11 input:[Group#8], table:t2", + "Group#8 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_10 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "select * from t t1 right join t t2 on t1.b = t2.b limit 1", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_5 input:[Group#2], offset:0, count:1", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Join_3 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.b, test.t.b)]", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_8 input:[Group#5], table:t1", + "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_7 table:t1, pk col:test.t.a", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_6 input:[Group#6], offset:0, count:1", + "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_10 input:[Group#7], table:t2", + "Group#7 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_11 input:[Group#8], offset:0, count:1", + "Group#8 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_9 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "select * from t t1 right join t t2 on t1.b = t2.b limit 5 offset 4", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_5 input:[Group#2], offset:4, count:5", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Join_3 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.b, test.t.b)]", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_8 input:[Group#5], table:t1", + "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_7 table:t1, pk col:test.t.a", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_6 input:[Group#6], offset:0, count:9", + "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_10 input:[Group#7], table:t2", + "Group#7 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_11 input:[Group#8], offset:0, count:9", + "Group#8 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_9 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "(select a from t) union all (select b from t) order by a limit 2;", + "Result": [ + "Group#0 Schema:[Column#25]", + " TopN_10 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_13 input:[Group#6], test.t.a:asc, offset:0, count:2", + " TopN_13 input:[Group#7], test.t.a:asc, offset:0, count:2", + " TopN_13 input:[Group#8], test.t.a:asc, offset:0, count:2", + " TopN_13 input:[Group#9], test.t.a:asc, offset:0, count:2", + " TopN_13 input:[Group#10], test.t.a:asc, offset:0, count:2", + " TopN_13 input:[Group#11], test.t.a:asc, offset:0, count:2", + " TopN_13 input:[Group#12], test.t.a:asc, offset:0, count:2", + "Group#6 Schema:[test.t.a]", + " TiKVSingleGather_19 input:[Group#13], table:t, index:c_d_e", + "Group#13 Schema:[test.t.a]", + " TopN_36 input:[Group#14], test.t.a:asc, offset:0, count:2", + "Group#14 Schema:[test.t.a]", + " IndexScan_18 table:t, index:c, d, e", + "Group#7 Schema:[test.t.a]", + " TiKVSingleGather_21 input:[Group#15], table:t, index:f", + "Group#15 Schema:[test.t.a]", + " TopN_35 input:[Group#16], test.t.a:asc, offset:0, count:2", + "Group#16 Schema:[test.t.a]", + " IndexScan_20 table:t, index:f", + "Group#8 Schema:[test.t.a]", + " TiKVSingleGather_23 input:[Group#17], table:t, index:g", + "Group#17 Schema:[test.t.a]", + " TopN_34 input:[Group#18], test.t.a:asc, offset:0, count:2", + "Group#18 Schema:[test.t.a]", + " IndexScan_22 table:t, index:g", + "Group#9 Schema:[test.t.a]", + " TiKVSingleGather_25 input:[Group#19], table:t, index:f_g", + "Group#19 Schema:[test.t.a]", + " TopN_33 input:[Group#20], test.t.a:asc, offset:0, count:2", + "Group#20 Schema:[test.t.a]", + " IndexScan_24 table:t, index:f, g", + "Group#10 Schema:[test.t.a]", + " TiKVSingleGather_27 input:[Group#21], table:t, index:c_d_e_str", + "Group#21 Schema:[test.t.a]", + " TopN_32 input:[Group#22], test.t.a:asc, offset:0, count:2", + "Group#22 Schema:[test.t.a]", + " IndexScan_26 table:t, index:c_str, d_str, e_str", + "Group#11 Schema:[test.t.a]", + " TiKVSingleGather_29 input:[Group#23], table:t, index:e_d_c_str_prefix", + "Group#23 Schema:[test.t.a]", + " TopN_31 input:[Group#24], test.t.a:asc, offset:0, count:2", + "Group#24 Schema:[test.t.a]", + " IndexScan_28 table:t, index:e_str, d_str, c_str", + "Group#12 Schema:[test.t.a]", + " TiKVSingleGather_17 input:[Group#25], table:t", + "Group#25 Schema:[test.t.a]", + " TopN_30 input:[Group#26], test.t.a:asc, offset:0, count:2", + "Group#26 Schema:[test.t.a]", + " TableScan_16 table:t, pk col:test.t.a", + "Group#3 Schema:[Column#25]", + " Projection_7 input:[Group#27], test.t.b", + "Group#27 Schema:[test.t.b]", + " Projection_2 input:[Group#28], test.t.b", + "Group#28 Schema:[test.t.b]", + " TopN_15 input:[Group#29], test.t.b:asc, offset:0, count:2", + "Group#29 Schema:[test.t.b]", + " TiKVSingleGather_38 input:[Group#30], table:t", + "Group#30 Schema:[test.t.b]", + " TopN_39 input:[Group#31], test.t.b:asc, offset:0, count:2", + "Group#31 Schema:[test.t.b]", + " TableScan_37 table:t" + ] + }, + { + "SQL": "(select a from t) union all (select b from t) limit 2;", + "Result": [ + "Group#0 Schema:[Column#25]", + " Limit_8 input:[Group#1], 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]", + " Limit_9 input:[Group#6], offset:0, count:2", + " Limit_9 input:[Group#7], offset:0, count:2", + " Limit_9 input:[Group#8], offset:0, count:2", + " Limit_9 input:[Group#9], offset:0, count:2", + " Limit_9 input:[Group#10], offset:0, count:2", + " Limit_9 input:[Group#11], offset:0, count:2", + " Limit_9 input:[Group#12], offset:0, count:2", + "Group#6 Schema:[test.t.a]", + " TiKVSingleGather_13 input:[Group#13], table:t, index:c_d_e", + "Group#13 Schema:[test.t.a]", + " Limit_30 input:[Group#14], offset:0, count:2", + "Group#14 Schema:[test.t.a]", + " IndexScan_12 table:t, index:c, d, e", + "Group#7 Schema:[test.t.a]", + " TiKVSingleGather_15 input:[Group#15], table:t, index:f", + "Group#15 Schema:[test.t.a]", + " Limit_29 input:[Group#16], offset:0, count:2", + "Group#16 Schema:[test.t.a]", + " IndexScan_14 table:t, index:f", + "Group#8 Schema:[test.t.a]", + " TiKVSingleGather_17 input:[Group#17], table:t, index:g", + "Group#17 Schema:[test.t.a]", + " Limit_28 input:[Group#18], offset:0, count:2", + "Group#18 Schema:[test.t.a]", + " IndexScan_16 table:t, index:g", + "Group#9 Schema:[test.t.a]", + " TiKVSingleGather_19 input:[Group#19], table:t, index:f_g", + "Group#19 Schema:[test.t.a]", + " Limit_27 input:[Group#20], offset:0, count:2", + "Group#20 Schema:[test.t.a]", + " IndexScan_18 table:t, index:f, g", + "Group#10 Schema:[test.t.a]", + " TiKVSingleGather_21 input:[Group#21], table:t, index:c_d_e_str", + "Group#21 Schema:[test.t.a]", + " Limit_26 input:[Group#22], offset:0, count:2", + "Group#22 Schema:[test.t.a]", + " IndexScan_20 table:t, index:c_str, d_str, e_str", + "Group#11 Schema:[test.t.a]", + " TiKVSingleGather_23 input:[Group#23], table:t, index:e_d_c_str_prefix", + "Group#23 Schema:[test.t.a]", + " Limit_25 input:[Group#24], offset:0, count:2", + "Group#24 Schema:[test.t.a]", + " IndexScan_22 table:t, index:e_str, d_str, c_str", + "Group#12 Schema:[test.t.a]", + " TiKVSingleGather_11 input:[Group#25], table:t", + "Group#25 Schema:[test.t.a]", + " Limit_24 input:[Group#26], offset:0, count:2", + "Group#26 Schema:[test.t.a]", + " TableScan_10 table:t, pk col:test.t.a", + "Group#3 Schema:[Column#25]", + " Projection_7 input:[Group#27], test.t.b", + "Group#27 Schema:[test.t.b]", + " Projection_2 input:[Group#28], test.t.b", + "Group#28 Schema:[test.t.b]", + " Limit_9 input:[Group#29], offset:0, count:2", + "Group#29 Schema:[test.t.b]", + " TiKVSingleGather_32 input:[Group#30], table:t", + "Group#30 Schema:[test.t.b]", + " Limit_33 input:[Group#31], offset:0, count:2", + "Group#31 Schema:[test.t.b]", + " TableScan_31 table:t" + ] + }, + { + "SQL": "(select a from t) union all (select b from t) limit 2 offset 5;", + "Result": [ + "Group#0 Schema:[Column#25]", + " Limit_8 input:[Group#1], offset:5, 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]", + " Limit_9 input:[Group#6], offset:0, count:7", + " Limit_9 input:[Group#7], offset:0, count:7", + " Limit_9 input:[Group#8], offset:0, count:7", + " Limit_9 input:[Group#9], offset:0, count:7", + " Limit_9 input:[Group#10], offset:0, count:7", + " Limit_9 input:[Group#11], offset:0, count:7", + " Limit_9 input:[Group#12], offset:0, count:7", + "Group#6 Schema:[test.t.a]", + " TiKVSingleGather_13 input:[Group#13], table:t, index:c_d_e", + "Group#13 Schema:[test.t.a]", + " Limit_30 input:[Group#14], offset:0, count:7", + "Group#14 Schema:[test.t.a]", + " IndexScan_12 table:t, index:c, d, e", + "Group#7 Schema:[test.t.a]", + " TiKVSingleGather_15 input:[Group#15], table:t, index:f", + "Group#15 Schema:[test.t.a]", + " Limit_29 input:[Group#16], offset:0, count:7", + "Group#16 Schema:[test.t.a]", + " IndexScan_14 table:t, index:f", + "Group#8 Schema:[test.t.a]", + " TiKVSingleGather_17 input:[Group#17], table:t, index:g", + "Group#17 Schema:[test.t.a]", + " Limit_28 input:[Group#18], offset:0, count:7", + "Group#18 Schema:[test.t.a]", + " IndexScan_16 table:t, index:g", + "Group#9 Schema:[test.t.a]", + " TiKVSingleGather_19 input:[Group#19], table:t, index:f_g", + "Group#19 Schema:[test.t.a]", + " Limit_27 input:[Group#20], offset:0, count:7", + "Group#20 Schema:[test.t.a]", + " IndexScan_18 table:t, index:f, g", + "Group#10 Schema:[test.t.a]", + " TiKVSingleGather_21 input:[Group#21], table:t, index:c_d_e_str", + "Group#21 Schema:[test.t.a]", + " Limit_26 input:[Group#22], offset:0, count:7", + "Group#22 Schema:[test.t.a]", + " IndexScan_20 table:t, index:c_str, d_str, e_str", + "Group#11 Schema:[test.t.a]", + " TiKVSingleGather_23 input:[Group#23], table:t, index:e_d_c_str_prefix", + "Group#23 Schema:[test.t.a]", + " Limit_25 input:[Group#24], offset:0, count:7", + "Group#24 Schema:[test.t.a]", + " IndexScan_22 table:t, index:e_str, d_str, c_str", + "Group#12 Schema:[test.t.a]", + " TiKVSingleGather_11 input:[Group#25], table:t", + "Group#25 Schema:[test.t.a]", + " Limit_24 input:[Group#26], offset:0, count:7", + "Group#26 Schema:[test.t.a]", + " TableScan_10 table:t, pk col:test.t.a", + "Group#3 Schema:[Column#25]", + " Projection_7 input:[Group#27], test.t.b", + "Group#27 Schema:[test.t.b]", + " Projection_2 input:[Group#28], test.t.b", + "Group#28 Schema:[test.t.b]", + " Limit_9 input:[Group#29], offset:0, count:7", + "Group#29 Schema:[test.t.b]", + " TiKVSingleGather_32 input:[Group#30], table:t", + "Group#30 Schema:[test.t.b]", + " Limit_33 input:[Group#31], offset:0, count:7", + "Group#31 Schema:[test.t.b]", + " TableScan_31 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_12 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, decimal(65,0) BINARY)->Column#26", + "Group#4 Schema:[test.t.a]", + " Projection_6 input:[Group#5], test.t.a", + "Group#5 Schema:[test.t.a]", + " TopN_15 input:[Group#6], cast(test.t.a):asc, offset:0, count:2", + " TopN_15 input:[Group#7], cast(test.t.a):asc, offset:0, count:2", + " TopN_15 input:[Group#8], cast(test.t.a):asc, offset:0, count:2", + " TopN_15 input:[Group#9], cast(test.t.a):asc, offset:0, count:2", + " TopN_15 input:[Group#10], cast(test.t.a):asc, offset:0, count:2", + " TopN_15 input:[Group#11], cast(test.t.a):asc, offset:0, count:2", + " TopN_15 input:[Group#12], cast(test.t.a):asc, offset:0, count:2", + "Group#6 Schema:[test.t.a]", + " TiKVSingleGather_21 input:[Group#13], table:t, index:c_d_e", + "Group#13 Schema:[test.t.a]", + " TopN_38 input:[Group#14], cast(test.t.a):asc, offset:0, count:2", + "Group#14 Schema:[test.t.a]", + " IndexScan_20 table:t, index:c, d, e", + "Group#7 Schema:[test.t.a]", + " TiKVSingleGather_23 input:[Group#15], table:t, index:f", + "Group#15 Schema:[test.t.a]", + " TopN_37 input:[Group#16], cast(test.t.a):asc, offset:0, count:2", + "Group#16 Schema:[test.t.a]", + " IndexScan_22 table:t, index:f", + "Group#8 Schema:[test.t.a]", + " TiKVSingleGather_25 input:[Group#17], table:t, index:g", + "Group#17 Schema:[test.t.a]", + " TopN_36 input:[Group#18], cast(test.t.a):asc, offset:0, count:2", + "Group#18 Schema:[test.t.a]", + " IndexScan_24 table:t, index:g", + "Group#9 Schema:[test.t.a]", + " TiKVSingleGather_27 input:[Group#19], table:t, index:f_g", + "Group#19 Schema:[test.t.a]", + " TopN_35 input:[Group#20], cast(test.t.a):asc, offset:0, count:2", + "Group#20 Schema:[test.t.a]", + " IndexScan_26 table:t, index:f, g", + "Group#10 Schema:[test.t.a]", + " TiKVSingleGather_29 input:[Group#21], table:t, index:c_d_e_str", + "Group#21 Schema:[test.t.a]", + " TopN_34 input:[Group#22], cast(test.t.a):asc, offset:0, count:2", + "Group#22 Schema:[test.t.a]", + " IndexScan_28 table:t, index:c_str, d_str, e_str", + "Group#11 Schema:[test.t.a]", + " TiKVSingleGather_31 input:[Group#23], table:t, index:e_d_c_str_prefix", + "Group#23 Schema:[test.t.a]", + " TopN_33 input:[Group#24], cast(test.t.a):asc, offset:0, count:2", + "Group#24 Schema:[test.t.a]", + " IndexScan_30 table:t, index:e_str, d_str, c_str", + "Group#12 Schema:[test.t.a]", + " TiKVSingleGather_19 input:[Group#25], table:t", + "Group#25 Schema:[test.t.a]", + " TopN_32 input:[Group#26], cast(test.t.a):asc, offset:0, count:2", + "Group#26 Schema:[test.t.a]", + " TableScan_18 table:t, pk col:test.t.a", + "Group#3 Schema:[Column#26]", + " Projection_9 input:[Group#27], cast(Column#13, decimal(65,0) BINARY)->Column#26", + "Group#27 Schema:[Column#13]", + " Projection_4 input:[Group#28], Column#13", + "Group#28 Schema:[Column#13]", + " TopN_17 input:[Group#29], cast(Column#13):asc, offset:0, count:2", + "Group#29 Schema:[Column#13]", + " Aggregation_3 input:[Group#30], group by:test.t.b, funcs:sum(test.t.a)", + "Group#30 Schema:[test.t.a,test.t.b]", + " Selection_2 input:[Group#31], gt(test.t.a, 2)", + "Group#31 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_40 input:[Group#32], table:t", + "Group#32 Schema:[test.t.a,test.t.b]", + " TableScan_39 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_12 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, decimal(65,0) BINARY)->Column#26", + "Group#4 Schema:[test.t.a]", + " Projection_6 input:[Group#5], test.t.a", + "Group#5 Schema:[test.t.a]", + " TopN_15 input:[Group#6], cast(test.t.a):asc, offset:0, count:3", + " TopN_15 input:[Group#7], cast(test.t.a):asc, offset:0, count:3", + " TopN_15 input:[Group#8], cast(test.t.a):asc, offset:0, count:3", + " TopN_15 input:[Group#9], cast(test.t.a):asc, offset:0, count:3", + " TopN_15 input:[Group#10], cast(test.t.a):asc, offset:0, count:3", + " TopN_15 input:[Group#11], cast(test.t.a):asc, offset:0, count:3", + " TopN_15 input:[Group#12], cast(test.t.a):asc, offset:0, count:3", + "Group#6 Schema:[test.t.a]", + " TiKVSingleGather_21 input:[Group#13], table:t, index:c_d_e", + "Group#13 Schema:[test.t.a]", + " TopN_38 input:[Group#14], cast(test.t.a):asc, offset:0, count:3", + "Group#14 Schema:[test.t.a]", + " IndexScan_20 table:t, index:c, d, e", + "Group#7 Schema:[test.t.a]", + " TiKVSingleGather_23 input:[Group#15], table:t, index:f", + "Group#15 Schema:[test.t.a]", + " TopN_37 input:[Group#16], cast(test.t.a):asc, offset:0, count:3", + "Group#16 Schema:[test.t.a]", + " IndexScan_22 table:t, index:f", + "Group#8 Schema:[test.t.a]", + " TiKVSingleGather_25 input:[Group#17], table:t, index:g", + "Group#17 Schema:[test.t.a]", + " TopN_36 input:[Group#18], cast(test.t.a):asc, offset:0, count:3", + "Group#18 Schema:[test.t.a]", + " IndexScan_24 table:t, index:g", + "Group#9 Schema:[test.t.a]", + " TiKVSingleGather_27 input:[Group#19], table:t, index:f_g", + "Group#19 Schema:[test.t.a]", + " TopN_35 input:[Group#20], cast(test.t.a):asc, offset:0, count:3", + "Group#20 Schema:[test.t.a]", + " IndexScan_26 table:t, index:f, g", + "Group#10 Schema:[test.t.a]", + " TiKVSingleGather_29 input:[Group#21], table:t, index:c_d_e_str", + "Group#21 Schema:[test.t.a]", + " TopN_34 input:[Group#22], cast(test.t.a):asc, offset:0, count:3", + "Group#22 Schema:[test.t.a]", + " IndexScan_28 table:t, index:c_str, d_str, e_str", + "Group#11 Schema:[test.t.a]", + " TiKVSingleGather_31 input:[Group#23], table:t, index:e_d_c_str_prefix", + "Group#23 Schema:[test.t.a]", + " TopN_33 input:[Group#24], cast(test.t.a):asc, offset:0, count:3", + "Group#24 Schema:[test.t.a]", + " IndexScan_30 table:t, index:e_str, d_str, c_str", + "Group#12 Schema:[test.t.a]", + " TiKVSingleGather_19 input:[Group#25], table:t", + "Group#25 Schema:[test.t.a]", + " TopN_32 input:[Group#26], cast(test.t.a):asc, offset:0, count:3", + "Group#26 Schema:[test.t.a]", + " TableScan_18 table:t, pk col:test.t.a", + "Group#3 Schema:[Column#26]", + " Projection_9 input:[Group#27], cast(Column#13, decimal(65,0) BINARY)->Column#26", + "Group#27 Schema:[Column#13]", + " Projection_4 input:[Group#28], Column#13", + "Group#28 Schema:[Column#13]", + " TopN_17 input:[Group#29], cast(Column#13):asc, offset:0, count:3", + "Group#29 Schema:[Column#13]", + " Aggregation_3 input:[Group#30], group by:test.t.b, funcs:sum(test.t.a)", + "Group#30 Schema:[test.t.a,test.t.b]", + " Selection_2 input:[Group#31], gt(test.t.a, 2)", + "Group#31 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_40 input:[Group#32], table:t", + "Group#32 Schema:[test.t.a,test.t.b]", + " TableScan_39 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_16 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_19 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_25 input:[Group#9], table:t", + " TiKVSingleGather_37 input:[Group#10], table:t, index:e_d_c_str_prefix", + " TiKVSingleGather_35 input:[Group#11], table:t, index:c_d_e_str", + " TiKVSingleGather_33 input:[Group#12], table:t, index:f_g", + " TiKVSingleGather_31 input:[Group#13], table:t, index:g", + " TiKVSingleGather_29 input:[Group#14], table:t, index:f", + " TiKVSingleGather_27 input:[Group#15], table:t, index:c_d_e", + "Group#9 Schema:[test.t.a]", + " TableScan_24 table:t, pk col:test.t.a", + "Group#10 Schema:[test.t.a]", + " IndexScan_36 table:t, index:e_str, d_str, c_str", + "Group#11 Schema:[test.t.a]", + " IndexScan_34 table:t, index:c_str, d_str, e_str", + "Group#12 Schema:[test.t.a]", + " IndexScan_32 table:t, index:f, g", + "Group#13 Schema:[test.t.a]", + " IndexScan_30 table:t, index:g", + "Group#14 Schema:[test.t.a]", + " IndexScan_28 table:t, index:f", + "Group#15 Schema:[test.t.a]", + " IndexScan_26 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_21 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_39 input:[Group#20], table:t", + "Group#20 Schema:[test.t.a,test.t.b]", + " TableScan_38 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_23 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_41 input:[Group#25], table:t", + " TiKVSingleGather_43 input:[Group#26], table:t, index:c_d_e", + "Group#25 Schema:[test.t.a,test.t.c]", + " TableScan_40 table:t, pk col:test.t.a", + "Group#26 Schema:[test.t.a,test.t.c]", + " IndexScan_42 table:t, index:c, d, e" + ] + } + ] + }, + { + "Name": "TestProjectionElimination", + "Cases": [ + { + "SQL": "select a, b from (select a, b from t) as t2", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select a+b from (select a, b from t) as t2", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], plus(test.t.a, test.t.b)->Column#13", + "Group#1 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select a from (select floor(a) as a from t) as t2", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_2 input:[Group#1], floor(test.t.a)->Column#13", + "Group#1 Schema:[test.t.a]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select a from (select a, b from (select a, b, c from t) as t2) as t3", + "Result": [ + "Group#0 Schema:[test.t.a]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select a+c from (select floor(a) as a, b, c from t) as t2", + "Result": [ + "Group#0 Schema:[Column#14]", + " Projection_4 input:[Group#1], plus(floor(test.t.a), test.t.c)->Column#14", + "Group#1 Schema:[test.t.a,test.t.c]", + " DataSource_1 table:t" + ] + } + ] + }, + { + "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]", + " DataSource_1 table:t", + "Group#3 Schema:[test.t.a]", + " 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]", + " DataSource_1 table:t", + "Group#3 Schema:[test.t.a]", + " TopN_4 input:[Group#2], test.t.a:asc, offset:0, count:1" + ] + } + ] + }, + { + "Name": "TestMergeAggregationProjection", + "Cases": [ + { + "SQL": "select b, max(a) from (select a, c+d as b from t as t1) as t2 group by b", + "Result": [ + "Group#0 Schema:[Column#13,Column#14]", + " Projection_4 input:[Group#1], Column#13, Column#14", + "Group#1 Schema:[Column#14,Column#13]", + " Aggregation_3 input:[Group#2], group by:Column#13, funcs:max(test.t.a), firstrow(Column#13)", + " Aggregation_5 input:[Group#3], group by:plus(test.t.c, test.t.d), funcs:max(test.t.a), firstrow(plus(test.t.c, test.t.d))", + "Group#2 Schema:[test.t.a,Column#13]", + " Projection_2 input:[Group#3], test.t.a, plus(test.t.c, test.t.d)->Column#13", + "Group#3 Schema:[test.t.a,test.t.c,test.t.d]", + " DataSource_1 table:t1" + ] + }, + { + "SQL": "select max(a) from (select c+d as b, a+c as a from t as t1) as t2", + "Result": [ + "Group#0 Schema:[Column#15]", + " Projection_4 input:[Group#1], Column#15", + "Group#1 Schema:[Column#15]", + " Aggregation_3 input:[Group#2], funcs:max(Column#14)", + " Aggregation_5 input:[Group#3], funcs:max(plus(test.t.a, test.t.c))", + "Group#2 Schema:[Column#14]", + " Projection_2 input:[Group#3], plus(test.t.a, test.t.c)->Column#14", + "Group#3 Schema:[test.t.a,test.t.c]", + " DataSource_1 table:t1" + ] + }, + { + "SQL": "select b, max(a) from (select a, c+d as b, @i:=0 from t as t1) as t2 group by b", + "Result": [ + "Group#0 Schema:[Column#13,Column#15]", + " Projection_4 input:[Group#1], Column#13, Column#15", + "Group#1 Schema:[Column#15,Column#13]", + " Aggregation_3 input:[Group#2], group by:Column#13, funcs:max(test.t.a), firstrow(Column#13)", + "Group#2 Schema:[test.t.a,Column#13,Column#14]", + " Projection_2 input:[Group#3], test.t.a, plus(test.t.c, test.t.d)->Column#13, setvar(i, 0)->Column#14", + "Group#3 Schema:[test.t.a,test.t.c,test.t.d]", + " DataSource_1 table:t1" + ] + } + ] + }, + { + "Name": "TestMergeAdjacentLimit", + "Cases": [ + { + "SQL": "select b from (select b from t limit 5) as t1 limit 10", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_4 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b]", + " Projection_2 input:[Group#2], test.t.b", + "Group#2 Schema:[test.t.b]", + " Limit_6 input:[Group#3], offset:0, count:5", + "Group#3 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select b from (select b from t limit 20) as t1 limit 10", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_4 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b]", + " Projection_2 input:[Group#2], test.t.b", + "Group#2 Schema:[test.t.b]", + " Limit_6 input:[Group#3], offset:0, count:10", + "Group#3 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select b from (select b from t limit 10) as t1 limit 10", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_4 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b]", + " Projection_2 input:[Group#2], test.t.b", + "Group#2 Schema:[test.t.b]", + " Limit_6 input:[Group#3], offset:0, count:10", + "Group#3 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select b from (select b from t limit 10 offset 10) as t1 limit 10 offset 5", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_4 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b]", + " Projection_2 input:[Group#2], test.t.b", + "Group#2 Schema:[test.t.b]", + " Limit_6 input:[Group#3], offset:15, count:5", + "Group#3 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select b from (select b from t limit 10 offset 2) as t1 limit 3 offset 5", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_4 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b]", + " Projection_2 input:[Group#2], test.t.b", + "Group#2 Schema:[test.t.b]", + " Limit_6 input:[Group#3], offset:7, count:3", + "Group#3 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select b from (select b from t limit 10 offset 5) as t1 limit 5 offset 5", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_4 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b]", + " Projection_2 input:[Group#2], test.t.b", + "Group#2 Schema:[test.t.b]", + " Limit_6 input:[Group#3], offset:10, count:5", + "Group#3 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select a from (select a from t limit 3 offset 5) t1 limit 3 offset 5", + "Result": [ + "Group#0 Schema:[test.t.a]", + " Projection_4 input:[Group#1], test.t.a", + "Group#1 Schema:[test.t.a]", + " Projection_2 input:[Group#2], test.t.a", + "Group#2 Schema:[test.t.a]", + " TableDual_6 rowcount:0" + ] + } + ] + }, + { + "Name": "TestMergeAdjacentTopN", + "Cases": [ + { + "SQL": "select b from (select b from t where c > 1 order by b limit 3) as t1 order by b limit 2", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_11 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b,test.t.c]", + " TopN_14 input:[Group#2], test.t.b:asc, offset:0, count:2", + "Group#2 Schema:[test.t.b,test.t.c]", + " Selection_2 input:[Group#3], gt(test.t.c, 1)", + "Group#3 Schema:[test.t.b,test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select a from (select a from t where b > 2 order by a limit 3 offset 1) as t1 order by a limit 2 offset 1", + "Result": [ + "Group#0 Schema:[test.t.a]", + " Projection_11 input:[Group#1], test.t.a", + "Group#1 Schema:[test.t.a,test.t.b]", + " TopN_14 input:[Group#2], test.t.a:asc, offset:2, count:2", + "Group#2 Schema:[test.t.a,test.t.b]", + " Selection_2 input:[Group#3], gt(test.t.b, 2)", + "Group#3 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select * from (select * from t order by a limit 3) as t1 order by a limit 5", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Projection_10 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TopN_13 input:[Group#2], test.t.a:asc, offset:0, count:3", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select b from (select b from t order by b limit 5) as t1 order by b limit 10", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_10 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b]", + " TopN_13 input:[Group#2], test.t.b:asc, offset:0, count:5", + "Group#2 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select b from (select b from t order by b limit 20) as t1 order by b limit 10", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_10 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b]", + " TopN_13 input:[Group#2], test.t.b:asc, offset:0, count:10", + "Group#2 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select b from (select b from t order by b limit 10) as t1 order by b limit 10", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_10 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b]", + " TopN_13 input:[Group#2], test.t.b:asc, offset:0, count:10", + "Group#2 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select b from (select b from t order by b limit 10 offset 10) as t1 order by b limit 10 offset 5", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_10 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b]", + " TopN_13 input:[Group#2], test.t.b:asc, offset:15, count:5", + "Group#2 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select b from (select b from t order by b limit 10 offset 2) as t1 order by b limit 3 offset 5", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_10 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b]", + " TopN_13 input:[Group#2], test.t.b:asc, offset:7, count:3", + "Group#2 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select b from (select b from t order by b limit 10 offset 5) as t1 order by b limit 5 offset 5", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_10 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.b]", + " TopN_13 input:[Group#2], test.t.b:asc, offset:10, count:5", + "Group#2 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select a from (select a from t order by a limit 3 offset 5) as t1 order by a limit 3 offset 5", + "Result": [ + "Group#0 Schema:[test.t.a]", + " Projection_10 input:[Group#1], test.t.a", + "Group#1 Schema:[test.t.a]", + " TableDual_13 rowcount:0" + ] + }, + { + "SQL": "select b from (select b from t where c > 1 order by b, a limit 3) as t1 order by b limit 2", + "Result": [ + "Group#0 Schema:[test.t.b]", + " Projection_13 input:[Group#1], test.t.b", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c]", + " TopN_16 input:[Group#2], test.t.b:asc, test.t.a:asc, offset:0, count:2", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", + " Selection_2 input:[Group#3], gt(test.t.c, 1)", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select a from (select a from t where b > 2 order by a, b limit 3 offset 1) as t1 order by a limit 2 offset 1", + "Result": [ + "Group#0 Schema:[test.t.a]", + " Projection_13 input:[Group#1], test.t.a", + "Group#1 Schema:[test.t.a,test.t.b]", + " TopN_16 input:[Group#2], test.t.a:asc, test.t.b:asc, offset:2, count:2", + "Group#2 Schema:[test.t.a,test.t.b]", + " Selection_2 input:[Group#3], gt(test.t.b, 2)", + "Group#3 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t" + ] + } + ] + }, + { + "Name": "TestTransformLimitToTableDual", + "Cases": [ + { + "SQL": "select a from t limit 0 offset 1", + "Result": [ + "Group#0 Schema:[test.t.a]", + " TableDual_4 rowcount:0" + ] + }, + { + "SQL": "select * from t as t1 left join (select * from t limit 0 offset 1) as t2 on t1.a = t2.a;", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Projection_6 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Join_5 input:[Group#2,Group#3], left outer join, equal:[eq(test.t.a, test.t.a)]", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " DataSource_1 table:t1", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableDual_7 rowcount:0" + ] + }, + { + "SQL": "select a, b from ((select a, b from t) union all(select c as a, d as b from t limit 0 offset 5)) as t1 where a > 1", + "Result": [ + "Group#0 Schema:[Column#25,Column#26]", + " Projection_10 input:[Group#1], Column#25, Column#26", + "Group#1 Schema:[Column#25,Column#26]", + " Selection_9 input:[Group#2], gt(Column#25, 1)", + "Group#2 Schema:[Column#25,Column#26]", + " Union_6 input:[Group#3,Group#4]", + "Group#3 Schema:[Column#25,Column#26]", + " Projection_7 input:[Group#5], test.t.a, test.t.b", + "Group#5 Schema:[test.t.a,test.t.b]", + " Projection_5 input:[Group#6], test.t.a, test.t.b", + "Group#6 Schema:[test.t.a,test.t.b]", + " DataSource_4 table:t", + "Group#4 Schema:[Column#25,Column#26]", + " Projection_8 input:[Group#7], test.t.c, test.t.d", + "Group#7 Schema:[test.t.c,test.t.d]", + " TableDual_11 rowcount:0" + ] + } + ] + }, + { + "Name": "TestPostTransformationRules", + "Cases": [ + { + "SQL": "select b from (select b+10 as b from t) as t1 order by b + 10 limit 10", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_8 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13,Column#14]", + " TopN_10 input:[Group#2], Column#14:asc, offset:0, count:10", + "Group#2 Schema:[Column#13,Column#14]", + " Projection_11 input:[Group#3], plus(test.t.b, 10)->Column#13, plus(plus(test.t.b, 10), 10)->Column#14", + "Group#3 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select * from (select a+1 as c, a+b as d from t) as t1 order by c+d limit 10", + "Result": [ + "Group#0 Schema:[Column#13,Column#14]", + " Projection_8 input:[Group#1], Column#13, Column#14", + "Group#1 Schema:[Column#13,Column#14,Column#15]", + " TopN_10 input:[Group#2], Column#15:asc, offset:0, count:10", + "Group#2 Schema:[Column#13,Column#14,Column#15]", + " Projection_11 input:[Group#3], plus(test.t.a, 1)->Column#13, plus(test.t.a, test.t.b)->Column#14, plus(plus(test.t.a, 1), plus(test.t.a, test.t.b))->Column#15", + "Group#3 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select a from (select a, b from t order by b limit 10) as t1", + "Result": [ + "Group#0 Schema:[test.t.a]", + " Projection_5 input:[Group#1], test.t.a", + "Group#1 Schema:[test.t.a,test.t.b]", + " TopN_6 input:[Group#2], test.t.b:asc, offset:0, count:10", + "Group#2 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t" + ] + } + ] + }, + { + "Name": "TestPushLimitDownTiKVSingleGather", + "Cases": [ + { + "SQL": "select * from t limit 1", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_3 input:[Group#1], offset:0, count:1", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_5 input:[Group#2], table:t", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_6 input:[Group#3], offset:0, count:1", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_4 table:t, pk col:test.t.a" + ] + }, + { + "SQL": "select * from t as t1 left join (select * from t limit 2) as t2 on t1.a = t2.a;", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Join_5 input:[Group#1,Group#2], left outer join, equal:[eq(test.t.a, test.t.a)]", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_8 input:[Group#3], table:t1", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_7 table:t1, pk col:test.t.a", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_4 input:[Group#4], offset:0, count:2", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TiKVSingleGather_10 input:[Group#5], table:t", + "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " Limit_11 input:[Group#6], offset:0, count:2", + "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", + " TableScan_9 table:t, pk col:test.t.a" + ] + }, + { + "SQL": "select a, b from ((select a, b from t) union all(select c as a, d as b from t limit 3)) as t1 where a > 1", + "Result": [ + "Group#0 Schema:[Column#25,Column#26]", + " Selection_9 input:[Group#1], gt(Column#25, 1)", + "Group#1 Schema:[Column#25,Column#26]", + " Union_6 input:[Group#2,Group#3]", + "Group#2 Schema:[Column#25,Column#26]", + " Projection_7 input:[Group#4], test.t.a, test.t.b", + "Group#4 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_12 input:[Group#5], table:t", + "Group#5 Schema:[test.t.a,test.t.b]", + " TableScan_11 table:t, pk col:test.t.a", + "Group#3 Schema:[Column#25,Column#26]", + " Projection_8 input:[Group#6], test.t.c, test.t.d", + "Group#6 Schema:[test.t.c,test.t.d]", + " Limit_3 input:[Group#7], offset:0, count:3", + " Limit_3 input:[Group#8], offset:0, count:3", + "Group#7 Schema:[test.t.c,test.t.d]", + " TiKVSingleGather_16 input:[Group#9], table:t, index:c_d_e", + "Group#9 Schema:[test.t.c,test.t.d]", + " Limit_18 input:[Group#10], offset:0, count:3", + "Group#10 Schema:[test.t.c,test.t.d]", + " IndexScan_15 table:t, index:c, d, e", + "Group#8 Schema:[test.t.c,test.t.d]", + " TiKVSingleGather_14 input:[Group#11], table:t", + "Group#11 Schema:[test.t.c,test.t.d]", + " Limit_17 input:[Group#12], offset:0, count:3", + "Group#12 Schema:[test.t.c,test.t.d]", + " TableScan_13 table:t" + ] + } + ] + }, + { + "Name": "TestEliminateOuterJoin", + "Cases": [ + { + "SQL": "select t1.a, max(t1.b) from t as t1 left join (select * from t) as t2 on t1.a = t2.a and t1.b = 3 group by t1.a;", + "Result": [ + "Group#0 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#1], test.t.a, Column#25", + "Group#1 Schema:[Column#25,test.t.a]", + " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", + "Group#2 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t1" + ] + }, + { + "SQL": "select t1.a, max(t1.b) from t as t1 left join (select * from t) as t2 on t1.a = t2.a group by t1.a;", + "Result": [ + "Group#0 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#1], test.t.a, Column#25", + "Group#1 Schema:[Column#25,test.t.a]", + " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", + "Group#2 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t1" + ] + }, + { + "SQL": "select t1.a, sum(t1.h) from t as t1 left join (select * from t) as t2 on t1.h = t2.h group by t1.a;", + "Result": [ + "Group#0 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#1], test.t.a, Column#25", + "Group#1 Schema:[Column#25,test.t.a]", + " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:sum(test.t.h), firstrow(test.t.a)", + "Group#2 Schema:[test.t.a,test.t.h]", + " Join_4 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.h, test.t.h)]", + "Group#3 Schema:[test.t.a,test.t.h]", + " DataSource_1 table:t1", + "Group#4 Schema:[test.t.h]", + " Projection_3 input:[Group#5], test.t.h", + "Group#5 Schema:[test.t.h]", + " DataSource_2 table:t" + ] + }, + { + "SQL": "select t1.a, sum(distinct t1.h) from t as t1 left join (select * from t) as t2 on t1.h = t2.h group by t1.a;", + "Result": [ + "Group#0 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#1], test.t.a, Column#25", + "Group#1 Schema:[Column#25,test.t.a]", + " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:sum(distinct test.t.h), firstrow(test.t.a)", + "Group#2 Schema:[test.t.a,test.t.h]", + " DataSource_1 table:t1" + ] + }, + { + "SQL": "select t1.a, max(t1.b) from t as t1 left join (select * from t) as t2 on t1.a = t2.a group by t2.a;", + "Result": [ + "Group#0 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#1], test.t.a, Column#25", + "Group#1 Schema:[Column#25,test.t.a]", + " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", + "Group#2 Schema:[test.t.a,test.t.b,test.t.a]", + " Join_4 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.a, test.t.a)]", + "Group#3 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t1", + "Group#4 Schema:[test.t.a]", + " Projection_3 input:[Group#5], test.t.a", + "Group#5 Schema:[test.t.a]", + " DataSource_2 table:t" + ] + }, + { + "SQL": "select t1.a, max(t2.b) from t as t1 left join (select * from t) as t2 on t1.a = t2.a group by t1.a;", + "Result": [ + "Group#0 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#1], test.t.a, Column#25", + "Group#1 Schema:[Column#25,test.t.a]", + " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", + "Group#2 Schema:[test.t.a,test.t.b]", + " Join_4 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.a, test.t.a)]", + "Group#3 Schema:[test.t.a]", + " DataSource_1 table:t1", + "Group#4 Schema:[test.t.a,test.t.b]", + " Projection_3 input:[Group#5], test.t.a, test.t.b", + "Group#5 Schema:[test.t.a,test.t.b]", + " DataSource_2 table:t" + ] + }, + { + "SQL": "select t2.a, max(t2.b) from t as t1 right join (select * from t) as t2 on t1.a = t2.a group by t2.a;", + "Result": [ + "Group#0 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#1], test.t.a, Column#25", + "Group#1 Schema:[Column#25,test.t.a]", + " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", + "Group#2 Schema:[test.t.a,test.t.b]", + " Projection_3 input:[Group#3], test.t.a, test.t.b", + "Group#3 Schema:[test.t.a,test.t.b]", + " DataSource_2 table:t" + ] + }, + { + "SQL": "select t2.a, max(t2.b) from t as t1 right join (select * from t) as t2 on t1.a = t2.a and t2.c = 3 group by t2.a;", + "Result": [ + "Group#0 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#1], test.t.a, Column#25", + "Group#1 Schema:[Column#25,test.t.a]", + " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", + " Projection_3 input:[Group#3], test.t.a, test.t.b, test.t.c", + "Group#3 Schema:[test.t.a,test.t.b,test.t.c]", + " DataSource_2 table:t" + ] + }, + { + "SQL": "select t2.a, sum(t2.h) from t as t1 right join (select * from t) as t2 on t1.h = t2.h group by t2.a;", + "Result": [ + "Group#0 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#1], test.t.a, Column#25", + "Group#1 Schema:[Column#25,test.t.a]", + " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:sum(test.t.h), firstrow(test.t.a)", + "Group#2 Schema:[test.t.a,test.t.h]", + " Join_4 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.h, test.t.h)]", + "Group#3 Schema:[test.t.h]", + " DataSource_1 table:t1", + "Group#4 Schema:[test.t.a,test.t.h]", + " Projection_3 input:[Group#5], test.t.a, test.t.h", + "Group#5 Schema:[test.t.a,test.t.h]", + " DataSource_2 table:t" + ] + }, + { + "SQL": "select t2.a, sum(distinct t2.h) from t as t1 right join (select * from t) as t2 on t1.h = t2.h group by t2.a;", + "Result": [ + "Group#0 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#1], test.t.a, Column#25", + "Group#1 Schema:[Column#25,test.t.a]", + " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:sum(distinct test.t.h), firstrow(test.t.a)", + "Group#2 Schema:[test.t.a,test.t.h]", + " Projection_3 input:[Group#3], test.t.a, test.t.h", + "Group#3 Schema:[test.t.a,test.t.h]", + " DataSource_2 table:t" + ] + }, + { + "SQL": "select t2.a, max(t2.b) from t as t1 right join (select * from t) as t2 on t1.a = t2.a group by t1.a;", + "Result": [ + "Group#0 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#1], test.t.a, Column#25", + "Group#1 Schema:[Column#25,test.t.a]", + " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", + "Group#2 Schema:[test.t.a,test.t.a,test.t.b]", + " Join_4 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.a, test.t.a)]", + "Group#3 Schema:[test.t.a]", + " DataSource_1 table:t1", + "Group#4 Schema:[test.t.a,test.t.b]", + " Projection_3 input:[Group#5], test.t.a, test.t.b", + "Group#5 Schema:[test.t.a,test.t.b]", + " DataSource_2 table:t" + ] + }, + { + "SQL": "select t2.a, max(t1.b) from t as t1 right join (select * from t) as t2 on t1.a = t2.a group by t2.a;", + "Result": [ + "Group#0 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#1], test.t.a, Column#25", + "Group#1 Schema:[Column#25,test.t.a]", + " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", + "Group#2 Schema:[test.t.b,test.t.a]", + " Join_4 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.a, test.t.a)]", + "Group#3 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t1", + "Group#4 Schema:[test.t.a]", + " Projection_3 input:[Group#5], test.t.a", + "Group#5 Schema:[test.t.a]", + " DataSource_2 table:t" + ] + }, + { + "SQL": "select t1.a, t1.b from t as t1 left join t as t2 on t1.a = t2.a and t1.b = 3 and t2.b = 3;", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b]", + " Projection_4 input:[Group#1], test.t.a, test.t.b", + "Group#1 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t1" + ] + }, + { + "SQL": "select t1.a, t1.b from t as t1 left join t as t2 on t1.b = t2.b and t1.a = 3 and t2.b = 3;", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b]", + " Projection_4 input:[Group#1], test.t.a, test.t.b", + "Group#1 Schema:[test.t.a,test.t.b]", + " Join_3 input:[Group#2,Group#3], left outer join, equal:[eq(test.t.b, test.t.b)], left cond:eq(test.t.a, 3), right cond:eq(test.t.b, 3)", + "Group#2 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t1", + "Group#3 Schema:[test.t.b]", + " DataSource_2 table:t2" + ] + }, + { + "SQL": "select t2.a, t1.b from t as t1 left join t as t2 on t1.a = t2.a and t1.a = 3 and t2.b = 3;", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b]", + " Projection_4 input:[Group#1], test.t.a, test.t.b", + "Group#1 Schema:[test.t.b,test.t.a]", + " Join_3 input:[Group#2,Group#3], left outer join, equal:[eq(test.t.a, test.t.a)], left cond:eq(test.t.a, 3), right cond:eq(test.t.b, 3)", + "Group#2 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t1", + "Group#3 Schema:[test.t.a,test.t.b]", + " DataSource_2 table:t2" + ] + }, + { + "SQL": "select t2.a, t2.b from t as t1 right join t as t2 on t1.a = t2.a and t1.b = 3 and t2.b = 3;", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b]", + " Projection_4 input:[Group#1], test.t.a, test.t.b", + "Group#1 Schema:[test.t.a,test.t.b]", + " DataSource_2 table:t2" + ] + }, + { + "SQL": "select t2.a, t2.b from t as t1 right join t as t2 on t1.b = t2.b and t1.a = 3 and t2.b = 3;", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b]", + " Projection_4 input:[Group#1], test.t.a, test.t.b", + "Group#1 Schema:[test.t.a,test.t.b]", + " Join_3 input:[Group#2,Group#3], right outer join, equal:[eq(test.t.b, test.t.b)], left cond:eq(test.t.a, 3), right cond:eq(test.t.b, 3)", + "Group#2 Schema:[test.t.a,test.t.b], UniqueKey:[test.t.a]", + " DataSource_1 table:t1", + "Group#3 Schema:[test.t.a,test.t.b]", + " DataSource_2 table:t2" + ] + }, + { + "SQL": "select t1.a, t2.b from t as t1 right join t as t2 on t1.a = t2.a and t1.a = 3 and t2.b = 3;", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b]", + " Projection_4 input:[Group#1], test.t.a, test.t.b", + "Group#1 Schema:[test.t.a,test.t.b]", + " Join_3 input:[Group#2,Group#3], right outer join, equal:[eq(test.t.a, test.t.a)], left cond:eq(test.t.a, 3), right cond:eq(test.t.b, 3)", + "Group#2 Schema:[test.t.a]", + " DataSource_1 table:t1", + "Group#3 Schema:[test.t.a,test.t.b]", + " DataSource_2 table:t2" + ] + }, + { + "SQL": "select t3.a, max(t3.b) from (select t1.a, t1.b from t as t1 left join t as t2 on t1.a = t2.a) t3 group by t3.a", + "Result": [ + "Group#0 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#1], test.t.a, Column#25", + "Group#1 Schema:[Column#25,test.t.a]", + " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", + "Group#2 Schema:[test.t.a,test.t.b]", + " Projection_4 input:[Group#3], test.t.a, test.t.b", + "Group#3 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t1" + ] + } + ] + }, + { + "Name": "TestTransformAggregateCaseToSelection", + "Cases": [ + { + "SQL": "select count(case when a > 10 then b end) from t", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_5 input:[Group#2], funcs:count(test.t.b)", + "Group#2 Schema:[test.t.a,test.t.b]", + " Selection_4 input:[Group#3], gt(test.t.a, 10)", + "Group#3 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(case when a > 10 then b end) from t group by c", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_2 input:[Group#2], group by:test.t.c, funcs:count(case(gt(test.t.a, 10), test.t.b))", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(case when a > 10 then b end) from t group by 'a'", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_2 input:[Group#2], group by:1, funcs:count(case(gt(test.t.a, 10), test.t.b))", + "Group#2 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(case when a > 10 then b end) from t group by concat(c, a)", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_2 input:[Group#2], group by:concat(cast(test.t.c), cast(test.t.a)), funcs:count(case(gt(test.t.a, 10), test.t.b))", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(case when a > 10 then b end) from t group by concat(c, 'a')", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_2 input:[Group#2], group by:concat(cast(test.t.c), \"a\"), funcs:count(case(gt(test.t.a, 10), test.t.b))", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(case when a > 10 then b end) from t group by concat('a', 'c')", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_2 input:[Group#2], group by:1, funcs:count(case(gt(test.t.a, 10), test.t.b))", + "Group#2 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(case when a > 10 then b else null end) from t", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_5 input:[Group#2], funcs:count(test.t.b)", + "Group#2 Schema:[test.t.a,test.t.b]", + " Selection_4 input:[Group#3], gt(test.t.a, 10)", + "Group#3 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(case when a > 10 then null else b end) from t", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_5 input:[Group#2], funcs:count(test.t.b)", + "Group#2 Schema:[test.t.a,test.t.b]", + " Selection_4 input:[Group#3], not(gt(test.t.a, 10))", + "Group#3 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(case when a > 10 then c else b end) 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:count(case(gt(test.t.a, 10), test.t.c, test.t.b))", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(case when a > 10 then c else 0 end) 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:count(case(gt(test.t.a, 10), test.t.c, 0))", + "Group#2 Schema:[test.t.a,test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select sum(case when a > 10 then c else 0 end) from t", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_5 input:[Group#2], funcs:sum(test.t.c)", + "Group#2 Schema:[test.t.a,test.t.c]", + " Selection_4 input:[Group#3], gt(test.t.a, 10)", + "Group#3 Schema:[test.t.a,test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select sum(case when a > 10 then c else 0.0 end) from t", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_5 input:[Group#2], funcs:sum(cast(test.t.c))", + "Group#2 Schema:[test.t.a,test.t.c]", + " Selection_4 input:[Group#3], gt(test.t.a, 10)", + "Group#3 Schema:[test.t.a,test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select sum(case when a > 10 then c else 1-1 end) from t", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_5 input:[Group#2], funcs:sum(test.t.c)", + "Group#2 Schema:[test.t.a,test.t.c]", + " Selection_4 input:[Group#3], gt(test.t.a, 10)", + "Group#3 Schema:[test.t.a,test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select sum(case when a > 0 then (case when a <= 1000 then b end) else 0 end) from t", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_7 input:[Group#2], funcs:sum(test.t.b)", + "Group#2 Schema:[test.t.a,test.t.b]", + " Selection_6 input:[Group#3], le(test.t.a, 1000)", + "Group#3 Schema:[test.t.a,test.t.b]", + " Selection_4 input:[Group#4], gt(test.t.a, 0)", + "Group#4 Schema:[test.t.a,test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select sum(case when a > 10 then 0 else c end) from t", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_5 input:[Group#2], funcs:sum(test.t.c)", + "Group#2 Schema:[test.t.a,test.t.c]", + " Selection_4 input:[Group#3], not(gt(test.t.a, 10))", + "Group#3 Schema:[test.t.a,test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select sum(case when a > 10 then 2 else 1 end) 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:sum(case(gt(test.t.a, 10), 2, 1))", + "Group#2 Schema:[test.t.a]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(DISTINCT case when a > 10 then null else c end) from t", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_5 input:[Group#2], funcs:count(distinct test.t.c)", + "Group#2 Schema:[test.t.a,test.t.c]", + " Selection_4 input:[Group#3], not(gt(test.t.a, 10))", + "Group#3 Schema:[test.t.a,test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select sum(DISTINCT case when a > 10 then c else 0 end) 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:sum(distinct case(gt(test.t.a, 10), test.t.c, 0))", + "Group#2 Schema:[test.t.a,test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select case when c > 10 then c end from t", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_2 input:[Group#1], case(gt(test.t.c, 10), test.t.c)->Column#13", + "Group#1 Schema:[test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(case when a > 10 then c end), c from t", + "Result": [ + "Group#0 Schema:[Column#13,test.t.c]", + " Projection_3 input:[Group#1], Column#13, test.t.c", + "Group#1 Schema:[Column#13,test.t.c]", + " Aggregation_2 input:[Group#2], funcs:count(case(gt(test.t.a, 10), test.t.c)), firstrow(test.t.c)", + "Group#2 Schema:[test.t.a,test.t.c]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(case when a > 10 and d < 5 then b end) from t", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_5 input:[Group#2], funcs:count(test.t.b)", + "Group#2 Schema:[test.t.a,test.t.b,test.t.d]", + " Selection_4 input:[Group#3], gt(test.t.a, 10), lt(test.t.d, 5)", + "Group#3 Schema:[test.t.a,test.t.b,test.t.d]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(case when a > 10 and d < 5 then null else b end) from t", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_5 input:[Group#2], funcs:count(test.t.b)", + "Group#2 Schema:[test.t.a,test.t.b,test.t.d]", + " Selection_4 input:[Group#3], not(and(gt(test.t.a, 10), lt(test.t.d, 5)))", + "Group#3 Schema:[test.t.a,test.t.b,test.t.d]", + " DataSource_1 table:t" + ] + } + ] + }, + { + "Name": "TestTransformAggToProj", + "Cases": [ + { + "SQL": "select count(b) from t group by a", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_5 input:[Group#1], 1->Column#13", + "Group#1 Schema:[test.t.a,test.t.b], UniqueKey:[test.t.a]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(b) from t group by b", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], Column#13", + "Group#1 Schema:[Column#13]", + " Aggregation_2 input:[Group#2], group by:test.t.b, funcs:count(test.t.b)", + "Group#2 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(b) 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:count(test.t.b)", + "Group#2 Schema:[test.t.b]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select a from t group by a having sum(b) > 4", + "Result": [ + "Group#0 Schema:[test.t.a]", + " Projection_5 input:[Group#1], test.t.a", + "Group#1 Schema:[test.t.a,Column#13]", + " Selection_4 input:[Group#2], gt(Column#13, 4)", + "Group#2 Schema:[test.t.a,Column#13]", + " Projection_7 input:[Group#3], test.t.a, cast(test.t.b, decimal(65,0) BINARY)->Column#13", + "Group#3 Schema:[test.t.a,test.t.b], UniqueKey:[test.t.a]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(b), sum(b), avg(b), b, max(b), min(b), bit_and(b), bit_or(b), bit_xor(b) from t group by a having sum(b) >= 0 and count(b) >= 0 order by b", + "Result": [ + "Group#0 Schema:[Column#21,Column#22,Column#23,test.t.b,Column#25,Column#26,Column#27,Column#28,Column#29]", + " Projection_6 input:[Group#1], Column#13, Column#14, Column#15, test.t.b, Column#16, Column#17, Column#18, Column#19, Column#20", + "Group#1 Schema:[Column#13,Column#14,Column#15,test.t.b,Column#16,Column#17,Column#18,Column#19,Column#20,Column#14,Column#13]", + " Sort_5 input:[Group#2], test.t.b:asc", + "Group#2 Schema:[Column#13,Column#14,Column#15,test.t.b,Column#16,Column#17,Column#18,Column#19,Column#20,Column#14,Column#13]", + " Selection_4 input:[Group#3], ge(Column#13, 0), ge(Column#14, 0)", + "Group#3 Schema:[Column#13,Column#14,Column#15,test.t.b,Column#16,Column#17,Column#18,Column#19,Column#20,Column#14,Column#13]", + " Projection_8 input:[Group#4], 1->Column#13, cast(test.t.b, decimal(65,0) BINARY)->Column#14, cast(test.t.b, decimal(65,30) BINARY)->Column#15, test.t.b, cast(test.t.b, int(11))->Column#16, cast(test.t.b, int(11))->Column#17, ifnull(cast(test.t.b, bigint(21) UNSIGNED BINARY), 18446744073709551615)->Column#18, ifnull(cast(test.t.b, bigint(21) UNSIGNED BINARY), 0)->Column#19, ifnull(cast(test.t.b, bigint(21) UNSIGNED BINARY), 0)->Column#20, cast(test.t.b, decimal(65,0) BINARY)->Column#14, 1->Column#13", + "Group#4 Schema:[test.t.a,test.t.b], UniqueKey:[test.t.a]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(b), sum(b), avg(b), f, max(c), min(c), bit_and(c), bit_or(d), bit_xor(g) from t group by a", + "Result": [ + "Group#0 Schema:[Column#13,Column#14,Column#15,test.t.f,Column#16,Column#17,Column#18,Column#19,Column#20]", + " Projection_5 input:[Group#1], 1->Column#13, cast(test.t.b, decimal(65,0) BINARY)->Column#14, cast(test.t.b, decimal(65,30) BINARY)->Column#15, test.t.f, cast(test.t.c, int(11))->Column#16, cast(test.t.c, int(11))->Column#17, ifnull(cast(test.t.c, bigint(21) UNSIGNED BINARY), 18446744073709551615)->Column#18, ifnull(cast(test.t.d, bigint(21) UNSIGNED BINARY), 0)->Column#19, ifnull(cast(test.t.g, bigint(21) UNSIGNED BINARY), 0)->Column#20", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.f,test.t.g], UniqueKey:[test.t.f,test.t.f,test.t.g,test.t.a]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(b), sum(b), avg(b), f, max(c), min(c), bit_and(c), bit_or(d), bit_xor(g), var_pop(b) from t group by a", + "Result": [ + "Group#0 Schema:[Column#13,Column#14,Column#15,test.t.f,Column#16,Column#17,Column#18,Column#19,Column#20,Column#21]", + " Projection_3 input:[Group#1], Column#13, Column#14, Column#15, test.t.f, Column#16, Column#17, Column#18, Column#19, Column#20, Column#21", + "Group#1 Schema:[Column#13,Column#14,Column#15,Column#16,Column#17,Column#18,Column#19,Column#20,Column#21,test.t.f]", + " Aggregation_2 input:[Group#2], group by:test.t.a, funcs:count(test.t.b), sum(test.t.b), avg(test.t.b), max(test.t.c), min(test.t.c), bit_and(test.t.c), bit_or(test.t.d), bit_xor(test.t.g), var_pop(test.t.b), firstrow(test.t.f)", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.f,test.t.g], UniqueKey:[test.t.f,test.t.f,test.t.g,test.t.a]", + " DataSource_1 table:t" + ] + }, + { + "SQL": "select count(b), sum(b), avg(b), f, max(c), min(c), bit_and(c), bit_or(d), bit_xor(g), group_concat(b, c, d, f) from t group by a", + "Result": [ + "Group#0 Schema:[Column#13,Column#14,Column#15,test.t.f,Column#16,Column#17,Column#18,Column#19,Column#20,Column#21]", + " Projection_3 input:[Group#1], Column#13, Column#14, Column#15, test.t.f, Column#16, Column#17, Column#18, Column#19, Column#20, Column#21", + "Group#1 Schema:[Column#13,Column#14,Column#15,Column#16,Column#17,Column#18,Column#19,Column#20,Column#21,test.t.f]", + " Aggregation_2 input:[Group#2], group by:test.t.a, funcs:count(test.t.b), sum(test.t.b), avg(test.t.b), max(test.t.c), min(test.t.c), bit_and(test.t.c), bit_or(test.t.d), bit_xor(test.t.g), group_concat(test.t.b, test.t.c, test.t.d, test.t.f separator \",\"), firstrow(test.t.f)", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.f,test.t.g]", + " DataSource_1 table:t" + ] + } + ] + }, + { + "Name": "TestDecorrelate", + "Cases": [ + { + "SQL": "select a from t t1 where exists (select 1 from t t2 where t1.a = t2.b)", + "Result": [ + "Group#0 Schema:[test.t.a]", + " Projection_7 input:[Group#1], test.t.a", + "Group#1 Schema:[test.t.a]", + " Apply_6 input:[Group#2,Group#3], semi join", + " Join_10 input:[Group#2,Group#4], semi join, equal:[eq(test.t.a, test.t.b)]", + "Group#2 Schema:[test.t.a]", + " DataSource_1 table:t1", + "Group#3 Schema:[test.t.b]", + " Selection_4 input:[Group#4], eq(test.t.a, test.t.b)", + "Group#4 Schema:[test.t.b]", + " DataSource_3 table:t2" + ] + } + ] + } +] diff --git a/planner/cascades/transformation_rules.go b/planner/cascades/transformation_rules.go index 4b394d4021793..1637637520139 100644 --- a/planner/cascades/transformation_rules.go +++ b/planner/cascades/transformation_rules.go @@ -16,6 +16,14 @@ package cascades import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/planner/memo" +<<<<<<< HEAD +======= + "github.com/pingcap/tidb/planner/util" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/set" +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) ) // Transformation defines the interface for the transformation rules. @@ -39,5 +47,2196 @@ var transformationMap = map[memo.Operand][]Transformation{ operandProject: []Transformation{ nil, }, +<<<<<<< HEAD */ +======= +} + +// PostTransformationBatch does the transformation which is related to +// the constraints of the execution engine of TiDB. +// For example, TopN/Sort only support `order by` columns in TiDB layer, +// as for scalar functions, we need to inject a Projection for them +// below the TopN/Sort. +var PostTransformationBatch = TransformationRuleBatch{ + memo.OperandProjection: { + NewRuleEliminateProjection(), + NewRuleMergeAdjacentProjection(), + }, + memo.OperandTopN: { + NewRuleInjectProjectionBelowTopN(), + }, +} + +type baseRule struct { + pattern *memo.Pattern +} + +// Match implements Transformation Interface. +func (r *baseRule) Match(expr *memo.ExprIter) bool { + return true +} + +// GetPattern implements Transformation Interface. +func (r *baseRule) GetPattern() *memo.Pattern { + return r.pattern +} + +// PushSelDownTableScan pushes the selection down to TableScan. +type PushSelDownTableScan struct { + baseRule +} + +// NewRulePushSelDownTableScan creates a new Transformation PushSelDownTableScan. +// The pattern of this rule is: `Selection -> TableScan` +func NewRulePushSelDownTableScan() Transformation { + rule := &PushSelDownTableScan{} + ts := memo.NewPattern(memo.OperandTableScan, memo.EngineTiKVOrTiFlash) + p := memo.BuildPattern(memo.OperandSelection, memo.EngineTiKVOrTiFlash, ts) + rule.pattern = p + return rule +} + +// OnTransform implements Transformation interface. +// +// It transforms `sel -> ts` to one of the following new exprs: +// 1. `newSel -> newTS` +// 2. `newTS` +// +// Filters of the old `sel` operator are removed if they are used to calculate +// the key ranges of the `ts` operator. +func (r *PushSelDownTableScan) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + ts := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalTableScan) + if ts.Handle == nil { + return nil, false, false, nil + } + accesses, remained := ranger.DetachCondsForColumn(ts.SCtx(), sel.Conditions, ts.Handle) + if accesses == nil { + return nil, false, false, nil + } + newTblScan := plannercore.LogicalTableScan{ + Source: ts.Source, + Handle: ts.Handle, + AccessConds: ts.AccessConds.Shallow(), + }.Init(ts.SCtx(), ts.SelectBlockOffset()) + newTblScan.AccessConds = append(newTblScan.AccessConds, accesses...) + tblScanExpr := memo.NewGroupExpr(newTblScan) + if len(remained) == 0 { + // `sel -> ts` is transformed to `newTS`. + return []*memo.GroupExpr{tblScanExpr}, true, false, nil + } + schema := old.GetExpr().Group.Prop.Schema + tblScanGroup := memo.NewGroupWithSchema(tblScanExpr, schema) + newSel := plannercore.LogicalSelection{Conditions: remained}.Init(sel.SCtx(), sel.SelectBlockOffset()) + selExpr := memo.NewGroupExpr(newSel) + selExpr.Children = append(selExpr.Children, tblScanGroup) + // `sel -> ts` is transformed to `newSel ->newTS`. + return []*memo.GroupExpr{selExpr}, true, false, nil +} + +// PushSelDownIndexScan pushes a Selection down to IndexScan. +type PushSelDownIndexScan struct { + baseRule +} + +// NewRulePushSelDownIndexScan creates a new Transformation PushSelDownIndexScan. +// The pattern of this rule is `Selection -> IndexScan`. +func NewRulePushSelDownIndexScan() Transformation { + rule := &PushSelDownIndexScan{} + rule.pattern = memo.BuildPattern( + memo.OperandSelection, + memo.EngineTiKVOnly, + memo.NewPattern(memo.OperandIndexScan, memo.EngineTiKVOnly), + ) + return rule +} + +// OnTransform implements Transformation interface. +// It will transform `Selection -> IndexScan` to: +// `IndexScan(with a new access range)` or +// `Selection -> IndexScan(with a new access range)` +// or just keep the two GroupExprs unchanged. +func (r *PushSelDownIndexScan) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + is := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalIndexScan) + if len(is.IdxCols) == 0 { + return nil, false, false, nil + } + conditions := sel.Conditions + if is.AccessConds != nil { + // If we have already pushed some conditions down here, + // we merge old AccessConds with new conditions, + // to make sure this rule can be applied more than once. + conditions = make([]expression.Expression, len(sel.Conditions)+len(is.AccessConds)) + copy(conditions, sel.Conditions) + copy(conditions[len(sel.Conditions):], is.AccessConds) + } + res, err := ranger.DetachCondAndBuildRangeForIndex(is.SCtx(), conditions, is.IdxCols, is.IdxColLens) + if err != nil { + return nil, false, false, err + } + if len(res.AccessConds) == len(is.AccessConds) { + // There is no condition can be pushed down as range, + // or the pushed down conditions are the same with before. + sameConds := true + for i := range res.AccessConds { + if !res.AccessConds[i].Equal(is.SCtx(), is.AccessConds[i]) { + sameConds = false + break + } + } + if sameConds { + return nil, false, false, nil + } + } + // TODO: `res` still has some unused fields: EqOrInCount, IsDNFCond. + newIs := plannercore.LogicalIndexScan{ + Source: is.Source, + IsDoubleRead: is.IsDoubleRead, + EqCondCount: res.EqCondCount, + AccessConds: res.AccessConds, + Ranges: res.Ranges, + Index: is.Index, + Columns: is.Columns, + FullIdxCols: is.FullIdxCols, + FullIdxColLens: is.FullIdxColLens, + IdxCols: is.IdxCols, + IdxColLens: is.IdxColLens, + }.Init(is.SCtx(), is.SelectBlockOffset()) + isExpr := memo.NewGroupExpr(newIs) + + if len(res.RemainedConds) == 0 { + return []*memo.GroupExpr{isExpr}, true, false, nil + } + isGroup := memo.NewGroupWithSchema(isExpr, old.Children[0].GetExpr().Group.Prop.Schema) + newSel := plannercore.LogicalSelection{Conditions: res.RemainedConds}.Init(sel.SCtx(), sel.SelectBlockOffset()) + selExpr := memo.NewGroupExpr(newSel) + selExpr.SetChildren(isGroup) + return []*memo.GroupExpr{selExpr}, true, false, nil +} + +// PushSelDownTiKVSingleGather pushes the selection down to child of TiKVSingleGather. +type PushSelDownTiKVSingleGather struct { + baseRule +} + +// NewRulePushSelDownTiKVSingleGather creates a new Transformation PushSelDownTiKVSingleGather. +// The pattern of this rule is `Selection -> TiKVSingleGather -> Any`. +func NewRulePushSelDownTiKVSingleGather() Transformation { + any := memo.NewPattern(memo.OperandAny, memo.EngineTiKVOrTiFlash) + tg := memo.BuildPattern(memo.OperandTiKVSingleGather, memo.EngineTiDBOnly, any) + p := memo.BuildPattern(memo.OperandSelection, memo.EngineTiDBOnly, tg) + + rule := &PushSelDownTiKVSingleGather{} + rule.pattern = p + return rule +} + +// OnTransform implements Transformation interface. +// +// It transforms `oldSel -> oldTg -> any` to one of the following new exprs: +// 1. `newTg -> pushedSel -> any` +// 2. `remainedSel -> newTg -> pushedSel -> any` +func (r *PushSelDownTiKVSingleGather) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + sg := old.Children[0].GetExpr().ExprNode.(*plannercore.TiKVSingleGather) + childGroup := old.Children[0].Children[0].Group + var pushed, remained []expression.Expression + sctx := sg.SCtx() + pushed, remained = expression.PushDownExprs(sctx.GetSessionVars().StmtCtx, sel.Conditions, sctx.GetClient(), kv.TiKV) + if len(pushed) == 0 { + return nil, false, false, nil + } + pushedSel := plannercore.LogicalSelection{Conditions: pushed}.Init(sctx, sel.SelectBlockOffset()) + pushedSelExpr := memo.NewGroupExpr(pushedSel) + pushedSelExpr.Children = append(pushedSelExpr.Children, childGroup) + pushedSelGroup := memo.NewGroupWithSchema(pushedSelExpr, childGroup.Prop.Schema).SetEngineType(childGroup.EngineType) + // The field content of TiKVSingleGather would not be modified currently, so we + // just reference the same tg instead of making a copy of it. + // + // TODO: if we save pushed filters later in TiKVSingleGather, in order to do partition + // pruning or skyline pruning, we need to make a copy of the TiKVSingleGather here. + tblGatherExpr := memo.NewGroupExpr(sg) + tblGatherExpr.Children = append(tblGatherExpr.Children, pushedSelGroup) + if len(remained) == 0 { + // `oldSel -> oldTg -> any` is transformed to `newTg -> pushedSel -> any`. + return []*memo.GroupExpr{tblGatherExpr}, true, false, nil + } + tblGatherGroup := memo.NewGroupWithSchema(tblGatherExpr, pushedSelGroup.Prop.Schema) + remainedSel := plannercore.LogicalSelection{Conditions: remained}.Init(sel.SCtx(), sel.SelectBlockOffset()) + remainedSelExpr := memo.NewGroupExpr(remainedSel) + remainedSelExpr.Children = append(remainedSelExpr.Children, tblGatherGroup) + // `oldSel -> oldTg -> any` is transformed to `remainedSel -> newTg -> pushedSel -> any`. + return []*memo.GroupExpr{remainedSelExpr}, true, false, nil +} + +// EnumeratePaths converts DataSource to table scan and index scans. +type EnumeratePaths struct { + baseRule +} + +// NewRuleEnumeratePaths creates a new Transformation EnumeratePaths. +// The pattern of this rule is: `DataSource`. +func NewRuleEnumeratePaths() Transformation { + rule := &EnumeratePaths{} + rule.pattern = memo.NewPattern(memo.OperandDataSource, memo.EngineTiDBOnly) + return rule +} + +// OnTransform implements Transformation interface. +func (r *EnumeratePaths) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + ds := old.GetExpr().ExprNode.(*plannercore.DataSource) + gathers := ds.Convert2Gathers() + for _, gather := range gathers { + expr := memo.Convert2GroupExpr(gather) + expr.Children[0].SetEngineType(memo.EngineTiKV) + newExprs = append(newExprs, expr) + } + return newExprs, true, false, nil +} + +// PushAggDownGather splits Aggregation to two stages, final and partial1, +// and pushed the partial Aggregation down to the child of TiKVSingleGather. +type PushAggDownGather struct { + baseRule +} + +// NewRulePushAggDownGather creates a new Transformation PushAggDownGather. +// The pattern of this rule is: `Aggregation -> TiKVSingleGather`. +func NewRulePushAggDownGather() Transformation { + rule := &PushAggDownGather{} + rule.pattern = memo.BuildPattern( + memo.OperandAggregation, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandTiKVSingleGather, memo.EngineTiDBOnly), + ) + return rule +} + +// Match implements Transformation interface. +func (r *PushAggDownGather) Match(expr *memo.ExprIter) bool { + if expr.GetExpr().HasAppliedRule(r) { + return false + } + agg := expr.GetExpr().ExprNode.(*plannercore.LogicalAggregation) + for _, aggFunc := range agg.AggFuncs { + if aggFunc.Mode != aggregation.CompleteMode { + return false + } + } + if agg.HasDistinct() { + // TODO: remove this logic after the cost estimation of distinct pushdown is implemented. + // If AllowDistinctAggPushDown is set to true, we should not consider RootTask. + if !agg.SCtx().GetSessionVars().AllowDistinctAggPushDown { + return false + } + } + childEngine := expr.Children[0].GetExpr().Children[0].EngineType + if childEngine != memo.EngineTiKV { + // TODO: Remove this check when we have implemented TiFlashAggregation. + return false + } + return plannercore.CheckAggCanPushCop(agg.SCtx(), agg.AggFuncs, agg.GroupByItems, kv.TiKV) +} + +// OnTransform implements Transformation interface. +// It will transform `Agg->Gather` to `Agg(Final) -> Gather -> Agg(Partial1)`. +func (r *PushAggDownGather) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + agg := old.GetExpr().ExprNode.(*plannercore.LogicalAggregation) + aggSchema := old.GetExpr().Group.Prop.Schema + gather := old.Children[0].GetExpr().ExprNode.(*plannercore.TiKVSingleGather) + childGroup := old.Children[0].GetExpr().Children[0] + // The old Aggregation should stay unchanged for other transformation. + // So we build a new LogicalAggregation for the partialAgg. + aggFuncs := make([]*aggregation.AggFuncDesc, len(agg.AggFuncs)) + for i := range agg.AggFuncs { + aggFuncs[i] = agg.AggFuncs[i].Clone() + } + gbyItems := make([]expression.Expression, len(agg.GroupByItems)) + copy(gbyItems, agg.GroupByItems) + + partialPref, finalPref, funcMap := plannercore.BuildFinalModeAggregation(agg.SCtx(), + &plannercore.AggInfo{ + AggFuncs: aggFuncs, + GroupByItems: gbyItems, + Schema: aggSchema, + }, true) + // Remove unnecessary FirstRow. + partialPref.AggFuncs = + plannercore.RemoveUnnecessaryFirstRow(agg.SCtx(), finalPref.AggFuncs, finalPref.GroupByItems, partialPref.AggFuncs, partialPref.GroupByItems, partialPref.Schema, funcMap) + + partialAgg := plannercore.LogicalAggregation{ + AggFuncs: partialPref.AggFuncs, + GroupByItems: partialPref.GroupByItems, + }.Init(agg.SCtx(), agg.SelectBlockOffset()) + partialAgg.CopyAggHints(agg) + + finalAgg := plannercore.LogicalAggregation{ + AggFuncs: finalPref.AggFuncs, + GroupByItems: finalPref.GroupByItems, + }.Init(agg.SCtx(), agg.SelectBlockOffset()) + finalAgg.CopyAggHints(agg) + + partialAggExpr := memo.NewGroupExpr(partialAgg) + partialAggExpr.SetChildren(childGroup) + partialAggGroup := memo.NewGroupWithSchema(partialAggExpr, partialPref.Schema).SetEngineType(childGroup.EngineType) + gatherExpr := memo.NewGroupExpr(gather) + gatherExpr.SetChildren(partialAggGroup) + gatherGroup := memo.NewGroupWithSchema(gatherExpr, partialPref.Schema) + finalAggExpr := memo.NewGroupExpr(finalAgg) + finalAggExpr.SetChildren(gatherGroup) + finalAggExpr.AddAppliedRule(r) + // We don't erase the old complete mode Aggregation because + // this transformation would not always be better. + return []*memo.GroupExpr{finalAggExpr}, false, false, nil +} + +// PushSelDownSort pushes the Selection down to the child of Sort. +type PushSelDownSort struct { + baseRule +} + +// NewRulePushSelDownSort creates a new Transformation PushSelDownSort. +// The pattern of this rule is: `Selection -> Sort`. +func NewRulePushSelDownSort() Transformation { + rule := &PushSelDownSort{} + rule.pattern = memo.BuildPattern( + memo.OperandSelection, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandSort, memo.EngineTiDBOnly), + ) + return rule +} + +// OnTransform implements Transformation interface. +// It will transform `sel->sort->x` to `sort->sel->x`. +func (r *PushSelDownSort) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + sort := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalSort) + childGroup := old.Children[0].GetExpr().Children[0] + + newSelExpr := memo.NewGroupExpr(sel) + newSelExpr.Children = append(newSelExpr.Children, childGroup) + newSelGroup := memo.NewGroupWithSchema(newSelExpr, childGroup.Prop.Schema) + + newSortExpr := memo.NewGroupExpr(sort) + newSortExpr.Children = append(newSortExpr.Children, newSelGroup) + return []*memo.GroupExpr{newSortExpr}, true, false, nil +} + +// PushSelDownProjection pushes the Selection down to the child of Projection. +type PushSelDownProjection struct { + baseRule +} + +// NewRulePushSelDownProjection creates a new Transformation PushSelDownProjection. +// The pattern of this rule is: `Selection -> Projection`. +func NewRulePushSelDownProjection() Transformation { + rule := &PushSelDownProjection{} + rule.pattern = memo.BuildPattern( + memo.OperandSelection, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandProjection, memo.EngineTiDBOnly), + ) + return rule +} + +// OnTransform implements Transformation interface. +// It will transform `selection -> projection -> x` to +// 1. `projection -> selection -> x` or +// 2. `selection -> projection -> selection -> x` or +// 3. just keep unchanged. +func (r *PushSelDownProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) + projSchema := old.Children[0].Prop.Schema + childGroup := old.Children[0].GetExpr().Children[0] + for _, expr := range proj.Exprs { + if expression.HasAssignSetVarFunc(expr) { + return nil, false, false, nil + } + } + canBePushed := make([]expression.Expression, 0, len(sel.Conditions)) + canNotBePushed := make([]expression.Expression, 0, len(sel.Conditions)) + for _, cond := range sel.Conditions { + if !expression.HasGetSetVarFunc(cond) { + canBePushed = append(canBePushed, expression.ColumnSubstitute(cond, projSchema, proj.Exprs)) + } else { + canNotBePushed = append(canNotBePushed, cond) + } + } + if len(canBePushed) == 0 { + return nil, false, false, nil + } + newBottomSel := plannercore.LogicalSelection{Conditions: canBePushed}.Init(sel.SCtx(), sel.SelectBlockOffset()) + newBottomSelExpr := memo.NewGroupExpr(newBottomSel) + newBottomSelExpr.SetChildren(childGroup) + newBottomSelGroup := memo.NewGroupWithSchema(newBottomSelExpr, childGroup.Prop.Schema) + newProjExpr := memo.NewGroupExpr(proj) + newProjExpr.SetChildren(newBottomSelGroup) + if len(canNotBePushed) == 0 { + return []*memo.GroupExpr{newProjExpr}, true, false, nil + } + newProjGroup := memo.NewGroupWithSchema(newProjExpr, projSchema) + newTopSel := plannercore.LogicalSelection{Conditions: canNotBePushed}.Init(sel.SCtx(), sel.SelectBlockOffset()) + newTopSelExpr := memo.NewGroupExpr(newTopSel) + newTopSelExpr.SetChildren(newProjGroup) + return []*memo.GroupExpr{newTopSelExpr}, true, false, nil +} + +// PushSelDownAggregation pushes Selection down to the child of Aggregation. +type PushSelDownAggregation struct { + baseRule +} + +// NewRulePushSelDownAggregation creates a new Transformation PushSelDownAggregation. +// The pattern of this rule is `Selection -> Aggregation`. +func NewRulePushSelDownAggregation() Transformation { + rule := &PushSelDownAggregation{} + rule.pattern = memo.BuildPattern( + memo.OperandSelection, + memo.EngineAll, + memo.NewPattern(memo.OperandAggregation, memo.EngineAll), + ) + return rule +} + +// OnTransform implements Transformation interface. +// It will transform `sel->agg->x` to `agg->sel->x` or `sel->agg->sel->x` +// or just keep the selection unchanged. +func (r *PushSelDownAggregation) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + agg := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalAggregation) + aggSchema := old.Children[0].Prop.Schema + var pushedExprs []expression.Expression + var remainedExprs []expression.Expression + exprsOriginal := make([]expression.Expression, 0, len(agg.AggFuncs)) + for _, aggFunc := range agg.AggFuncs { + exprsOriginal = append(exprsOriginal, aggFunc.Args[0]) + } + groupByColumns := expression.NewSchema(agg.GetGroupByCols()...) + for _, cond := range sel.Conditions { + switch cond.(type) { + case *expression.Constant: + // Consider SQL list "select sum(b) from t group by a having 1=0". "1=0" is a constant predicate which should be + // retained and pushed down at the same time. Because we will get a wrong query result that contains one column + // with value 0 rather than an empty query result. + pushedExprs = append(pushedExprs, cond) + remainedExprs = append(remainedExprs, cond) + case *expression.ScalarFunction: + extractedCols := expression.ExtractColumns(cond) + canPush := true + for _, col := range extractedCols { + if !groupByColumns.Contains(col) { + canPush = false + break + } + } + if canPush { + pushedExprs = append(pushedExprs, cond) + } else { + remainedExprs = append(remainedExprs, cond) + } + default: + remainedExprs = append(remainedExprs, cond) + } + } + // If no condition can be pushed, keep the selection unchanged. + if len(pushedExprs) == 0 { + return nil, false, false, nil + } + sctx := sel.SCtx() + childGroup := old.Children[0].GetExpr().Children[0] + pushedSel := plannercore.LogicalSelection{Conditions: pushedExprs}.Init(sctx, sel.SelectBlockOffset()) + pushedGroupExpr := memo.NewGroupExpr(pushedSel) + pushedGroupExpr.SetChildren(childGroup) + pushedGroup := memo.NewGroupWithSchema(pushedGroupExpr, childGroup.Prop.Schema) + + aggGroupExpr := memo.NewGroupExpr(agg) + aggGroupExpr.SetChildren(pushedGroup) + + if len(remainedExprs) == 0 { + return []*memo.GroupExpr{aggGroupExpr}, true, false, nil + } + + aggGroup := memo.NewGroupWithSchema(aggGroupExpr, aggSchema) + remainedSel := plannercore.LogicalSelection{Conditions: remainedExprs}.Init(sctx, sel.SelectBlockOffset()) + remainedGroupExpr := memo.NewGroupExpr(remainedSel) + remainedGroupExpr.SetChildren(aggGroup) + return []*memo.GroupExpr{remainedGroupExpr}, true, false, nil +} + +// PushSelDownWindow pushes Selection down to the child of Window. +type PushSelDownWindow struct { + baseRule +} + +// NewRulePushSelDownWindow creates a new Transformation PushSelDownWindow. +// The pattern of this rule is `Selection -> Window`. +func NewRulePushSelDownWindow() Transformation { + rule := &PushSelDownWindow{} + rule.pattern = memo.BuildPattern( + memo.OperandSelection, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandWindow, memo.EngineAll), + ) + return rule +} + +// OnTransform implements Transformation interface. +// This rule will transform `sel -> window -> x` to +// 1. `window -> sel -> x` or +// 2. `sel -> window -> sel -> x` or +// 3. just keep unchanged. +func (r *PushSelDownWindow) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + window := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalWindow) + windowSchema := old.Children[0].Prop.Schema + childGroup := old.Children[0].GetExpr().Children[0] + canBePushed := make([]expression.Expression, 0, len(sel.Conditions)) + canNotBePushed := make([]expression.Expression, 0, len(sel.Conditions)) + + // get partition Columns' Schema + partitionColsSchema := expression.NewSchema(window.GetPartitionByCols()...) + + for _, cond := range sel.Conditions { + if expression.ExprFromSchema(cond, partitionColsSchema) { + canBePushed = append(canBePushed, cond) + } else { + canNotBePushed = append(canNotBePushed, cond) + } + } + // Nothing can be pushed! + if len(canBePushed) == 0 { + return nil, false, false, nil + } + + // construct return GroupExpr + newBottomSel := plannercore.LogicalSelection{Conditions: canBePushed}.Init(sel.SCtx(), sel.SelectBlockOffset()) + newBottomSelExpr := memo.NewGroupExpr(newBottomSel) + newBottomSelExpr.SetChildren(childGroup) + newBottomSelGroup := memo.NewGroupWithSchema(newBottomSelExpr, childGroup.Prop.Schema) + newWindowExpr := memo.NewGroupExpr(window) + newWindowExpr.SetChildren(newBottomSelGroup) + if len(canNotBePushed) == 0 { + return []*memo.GroupExpr{newWindowExpr}, true, false, nil + } + + newWindowGroup := memo.NewGroupWithSchema(newWindowExpr, windowSchema) + newTopSel := plannercore.LogicalSelection{Conditions: canNotBePushed}.Init(sel.SCtx(), sel.SelectBlockOffset()) + newTopSelExpr := memo.NewGroupExpr(newTopSel) + newTopSelExpr.SetChildren(newWindowGroup) + return []*memo.GroupExpr{newTopSelExpr}, true, false, nil +} + +// TransformLimitToTopN transforms Limit+Sort to TopN. +type TransformLimitToTopN struct { + baseRule +} + +// NewRuleTransformLimitToTopN creates a new Transformation TransformLimitToTopN. +// The pattern of this rule is `Limit -> Sort`. +func NewRuleTransformLimitToTopN() Transformation { + rule := &TransformLimitToTopN{} + rule.pattern = memo.BuildPattern( + memo.OperandLimit, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandSort, memo.EngineTiDBOnly), + ) + return rule +} + +// 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 +} + +// PushLimitDownProjection pushes Limit to Projection. +type PushLimitDownProjection struct { + baseRule +} + +// NewRulePushLimitDownProjection creates a new Transformation. +// The pattern of this rule is `Limit->Projection->X` to `Projection->Limit->X`. +func NewRulePushLimitDownProjection() Transformation { + rule := &PushLimitDownProjection{} + rule.pattern = memo.BuildPattern( + memo.OperandLimit, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandProjection, memo.EngineTiDBOnly), + ) + return rule +} + +// Match implements Transformation interface. +func (r *PushLimitDownProjection) Match(expr *memo.ExprIter) bool { + proj := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) + for _, expr := range proj.Exprs { + if expression.HasAssignSetVarFunc(expr) { + return false + } + } + return true +} + +// OnTransform implements Transformation interface. +// This rule tries to pushes the Limit through Projection. +func (r *PushLimitDownProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + limit := old.GetExpr().ExprNode.(*plannercore.LogicalLimit) + proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) + childGroup := old.Children[0].GetExpr().Children[0] + + projExpr := memo.NewGroupExpr(proj) + limitExpr := memo.NewGroupExpr(limit) + limitExpr.SetChildren(childGroup) + limitGroup := memo.NewGroupWithSchema(limitExpr, childGroup.Prop.Schema) + projExpr.SetChildren(limitGroup) + return []*memo.GroupExpr{projExpr}, true, false, nil +} + +// PushLimitDownUnionAll pushes limit to union all. +type PushLimitDownUnionAll struct { + baseRule +} + +// NewRulePushLimitDownUnionAll creates a new Transformation PushLimitDownUnionAll. +// The pattern of this rule is `Limit->UnionAll->X`. +func NewRulePushLimitDownUnionAll() Transformation { + rule := &PushLimitDownUnionAll{} + rule.pattern = memo.BuildPattern( + memo.OperandLimit, + 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 *PushLimitDownUnionAll) Match(expr *memo.ExprIter) bool { + return !expr.GetExpr().HasAppliedRule(r) +} + +// OnTransform implements Transformation interface. +// It will transform `Limit->UnionAll->X` to `Limit->UnionAll->Limit->X`. +func (r *PushLimitDownUnionAll) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + limit := old.GetExpr().ExprNode.(*plannercore.LogicalLimit) + unionAll := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalUnionAll) + unionAllSchema := old.Children[0].Group.Prop.Schema + + newLimit := plannercore.LogicalLimit{ + Count: limit.Count + limit.Offset, + }.Init(limit.SCtx(), limit.SelectBlockOffset()) + + newUnionAllExpr := memo.NewGroupExpr(unionAll) + for _, childGroup := range old.Children[0].GetExpr().Children { + newLimitExpr := memo.NewGroupExpr(newLimit) + newLimitExpr.Children = append(newLimitExpr.Children, childGroup) + newLimitGroup := memo.NewGroupWithSchema(newLimitExpr, childGroup.Prop.Schema) + + newUnionAllExpr.Children = append(newUnionAllExpr.Children, newLimitGroup) + } + + newLimitExpr := memo.NewGroupExpr(limit) + newUnionAllGroup := memo.NewGroupWithSchema(newUnionAllExpr, unionAllSchema) + newLimitExpr.SetChildren(newUnionAllGroup) + newLimitExpr.AddAppliedRule(r) + return []*memo.GroupExpr{newLimitExpr}, true, false, nil +} + +// PushSelDownJoin pushes Selection through Join. +type PushSelDownJoin struct { + baseRule +} + +// NewRulePushSelDownJoin creates a new Transformation PushSelDownJoin. +// The pattern of this rule is `Selection -> Join`. +func NewRulePushSelDownJoin() Transformation { + rule := &PushSelDownJoin{} + rule.pattern = memo.BuildPattern( + memo.OperandSelection, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandJoin, memo.EngineTiDBOnly), + ) + return rule +} + +// Match implements Transformation interface. +func (r *PushSelDownJoin) Match(expr *memo.ExprIter) bool { + return !expr.GetExpr().HasAppliedRule(r) +} + +// buildChildSelectionGroup builds a new childGroup if the pushed down condition is not empty. +func buildChildSelectionGroup( + oldSel *plannercore.LogicalSelection, + conditions []expression.Expression, + childGroup *memo.Group) *memo.Group { + if len(conditions) == 0 { + return childGroup + } + newSel := plannercore.LogicalSelection{Conditions: conditions}.Init(oldSel.SCtx(), oldSel.SelectBlockOffset()) + groupExpr := memo.NewGroupExpr(newSel) + groupExpr.SetChildren(childGroup) + newChild := memo.NewGroupWithSchema(groupExpr, childGroup.Prop.Schema) + return newChild +} + +// OnTransform implements Transformation interface. +// This rule tries to pushes the Selection through Join. Besides, this rule fulfills the `XXXConditions` field of Join. +func (r *PushSelDownJoin) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + joinExpr := old.Children[0].GetExpr() + // TODO: we need to create a new LogicalJoin here. + join := joinExpr.ExprNode.(*plannercore.LogicalJoin) + sctx := sel.SCtx() + leftGroup := old.Children[0].GetExpr().Children[0] + rightGroup := old.Children[0].GetExpr().Children[1] + var equalCond []*expression.ScalarFunction + var leftPushCond, rightPushCond, otherCond, leftCond, rightCond, remainCond []expression.Expression + switch join.JoinType { + case plannercore.SemiJoin, plannercore.InnerJoin: + tempCond := make([]expression.Expression, 0, + len(join.LeftConditions)+len(join.RightConditions)+len(join.EqualConditions)+len(join.OtherConditions)+len(sel.Conditions)) + tempCond = append(tempCond, join.LeftConditions...) + tempCond = append(tempCond, join.RightConditions...) + tempCond = append(tempCond, expression.ScalarFuncs2Exprs(join.EqualConditions)...) + tempCond = append(tempCond, join.OtherConditions...) + tempCond = append(tempCond, sel.Conditions...) + tempCond = expression.ExtractFiltersFromDNFs(sctx, tempCond) + tempCond = expression.PropagateConstant(sctx, tempCond) + // Return table dual when filter is constant false or null. + dual := plannercore.Conds2TableDual(join, tempCond) + if dual != nil { + return []*memo.GroupExpr{memo.NewGroupExpr(dual)}, false, true, nil + } + equalCond, leftPushCond, rightPushCond, otherCond = join.ExtractOnCondition(tempCond, leftGroup.Prop.Schema, rightGroup.Prop.Schema, true, true) + join.LeftConditions = nil + join.RightConditions = nil + join.EqualConditions = equalCond + join.OtherConditions = otherCond + leftCond = leftPushCond + rightCond = rightPushCond + case plannercore.LeftOuterJoin, plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin, + plannercore.RightOuterJoin: + lenJoinConds := len(join.EqualConditions) + len(join.LeftConditions) + len(join.RightConditions) + len(join.OtherConditions) + joinConds := make([]expression.Expression, 0, lenJoinConds) + for _, equalCond := range join.EqualConditions { + joinConds = append(joinConds, equalCond) + } + joinConds = append(joinConds, join.LeftConditions...) + joinConds = append(joinConds, join.RightConditions...) + joinConds = append(joinConds, join.OtherConditions...) + join.EqualConditions = nil + join.LeftConditions = nil + join.RightConditions = nil + join.OtherConditions = nil + remainCond = make([]expression.Expression, len(sel.Conditions)) + copy(remainCond, sel.Conditions) + nullSensitive := join.JoinType == plannercore.AntiLeftOuterSemiJoin || join.JoinType == plannercore.LeftOuterSemiJoin + if join.JoinType == plannercore.RightOuterJoin { + joinConds, remainCond = expression.PropConstOverOuterJoin(join.SCtx(), joinConds, remainCond, rightGroup.Prop.Schema, leftGroup.Prop.Schema, nullSensitive) + } else { + joinConds, remainCond = expression.PropConstOverOuterJoin(join.SCtx(), joinConds, remainCond, leftGroup.Prop.Schema, rightGroup.Prop.Schema, nullSensitive) + } + eq, left, right, other := join.ExtractOnCondition(joinConds, leftGroup.Prop.Schema, rightGroup.Prop.Schema, false, false) + join.AppendJoinConds(eq, left, right, other) + // Return table dual when filter is constant false or null. + dual := plannercore.Conds2TableDual(join, remainCond) + if dual != nil { + return []*memo.GroupExpr{memo.NewGroupExpr(dual)}, false, true, nil + } + if join.JoinType == plannercore.RightOuterJoin { + remainCond = expression.ExtractFiltersFromDNFs(join.SCtx(), remainCond) + // Only derive right where condition, because left where condition cannot be pushed down + equalCond, leftPushCond, rightPushCond, otherCond = join.ExtractOnCondition(remainCond, leftGroup.Prop.Schema, rightGroup.Prop.Schema, false, true) + rightCond = rightPushCond + // Handle join conditions, only derive left join condition, because right join condition cannot be pushed down + derivedLeftJoinCond, _ := plannercore.DeriveOtherConditions(join, true, false) + leftCond = append(join.LeftConditions, derivedLeftJoinCond...) + join.LeftConditions = nil + remainCond = append(expression.ScalarFuncs2Exprs(equalCond), otherCond...) + remainCond = append(remainCond, leftPushCond...) + } else { + remainCond = expression.ExtractFiltersFromDNFs(join.SCtx(), remainCond) + // Only derive left where condition, because right where condition cannot be pushed down + equalCond, leftPushCond, rightPushCond, otherCond = join.ExtractOnCondition(remainCond, leftGroup.Prop.Schema, rightGroup.Prop.Schema, true, false) + leftCond = leftPushCond + // Handle join conditions, only derive left join condition, because right join condition cannot be pushed down + _, derivedRightJoinCond := plannercore.DeriveOtherConditions(join, false, true) + rightCond = append(join.RightConditions, derivedRightJoinCond...) + join.RightConditions = nil + remainCond = append(expression.ScalarFuncs2Exprs(equalCond), otherCond...) + remainCond = append(remainCond, rightPushCond...) + } + default: + // TODO: Enhance this rule to deal with Semi/SmiAnti Joins. + } + leftCond = expression.RemoveDupExprs(sctx, leftCond) + rightCond = expression.RemoveDupExprs(sctx, rightCond) + // TODO: Update EqualConditions like what we have done in the method join.updateEQCond() before. + leftGroup = buildChildSelectionGroup(sel, leftCond, leftGroup) + rightGroup = buildChildSelectionGroup(sel, rightCond, rightGroup) + newJoinExpr := memo.NewGroupExpr(join) + newJoinExpr.SetChildren(leftGroup, rightGroup) + if len(remainCond) > 0 { + newSel := plannercore.LogicalSelection{Conditions: remainCond}.Init(sctx, sel.SelectBlockOffset()) + newSel.Conditions = remainCond + newSelExpr := memo.NewGroupExpr(newSel) + newSelExpr.SetChildren(memo.NewGroupWithSchema(newJoinExpr, old.Children[0].Prop.Schema)) + newSelExpr.AddAppliedRule(r) + return []*memo.GroupExpr{newSelExpr}, true, false, nil + } + return []*memo.GroupExpr{newJoinExpr}, true, false, nil +} + +// PushSelDownUnionAll pushes selection through union all. +type PushSelDownUnionAll struct { + baseRule +} + +// NewRulePushSelDownUnionAll creates a new Transformation PushSelDownUnionAll. +// The pattern of this rule is `Selection -> UnionAll`. +func NewRulePushSelDownUnionAll() Transformation { + rule := &PushSelDownUnionAll{} + rule.pattern = memo.BuildPattern( + memo.OperandSelection, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandUnionAll, memo.EngineTiDBOnly), + ) + return rule +} + +// OnTransform implements Transformation interface. +// It will transform `Selection->UnionAll->x` to `UnionAll->Selection->x`. +func (r *PushSelDownUnionAll) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + unionAll := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalUnionAll) + childGroups := old.Children[0].GetExpr().Children + + newUnionAllExpr := memo.NewGroupExpr(unionAll) + for _, group := range childGroups { + newSelExpr := memo.NewGroupExpr(sel) + newSelExpr.Children = append(newSelExpr.Children, group) + newSelGroup := memo.NewGroupWithSchema(newSelExpr, group.Prop.Schema) + + newUnionAllExpr.Children = append(newUnionAllExpr.Children, newSelGroup) + } + return []*memo.GroupExpr{newUnionAllExpr}, true, false, nil +} + +// EliminateProjection eliminates the projection. +type EliminateProjection struct { + baseRule +} + +// NewRuleEliminateProjection creates a new Transformation EliminateProjection. +// The pattern of this rule is `Projection -> Any`. +func NewRuleEliminateProjection() Transformation { + rule := &EliminateProjection{} + rule.pattern = memo.BuildPattern( + memo.OperandProjection, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandAny, memo.EngineTiDBOnly), + ) + return rule +} + +// OnTransform implements Transformation interface. +// This rule tries to eliminate the projection whose output columns are the same with its child. +func (r *EliminateProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + child := old.Children[0] + if child.Group.Prop.Schema.Len() != old.GetExpr().Group.Prop.Schema.Len() { + return nil, false, false, nil + } + + oldCols := old.GetExpr().Group.Prop.Schema.Columns + for i, col := range child.Group.Prop.Schema.Columns { + if !col.Equal(nil, oldCols[i]) { + return nil, false, false, nil + } + } + + // Promote the children group's expression. + finalGroupExprs := make([]*memo.GroupExpr, 0, child.Group.Equivalents.Len()) + for elem := child.Group.Equivalents.Front(); elem != nil; elem = elem.Next() { + childExpr := elem.Value.(*memo.GroupExpr) + copyChildExpr := memo.NewGroupExpr(childExpr.ExprNode) + copyChildExpr.SetChildren(childExpr.Children...) + finalGroupExprs = append(finalGroupExprs, copyChildExpr) + } + return finalGroupExprs, true, false, nil +} + +// MergeAdjacentProjection merge the adjacent projection. +type MergeAdjacentProjection struct { + baseRule +} + +// NewRuleMergeAdjacentProjection creates a new Transformation MergeAdjacentProjection. +// The pattern of this rule is `Projection -> Projection`. +func NewRuleMergeAdjacentProjection() Transformation { + rule := &MergeAdjacentProjection{} + rule.pattern = memo.BuildPattern( + memo.OperandProjection, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandProjection, memo.EngineTiDBOnly), + ) + return rule +} + +// OnTransform implements Transformation interface. +// It will transform `proj->proj->x` to `proj->x` +// or just keep the adjacent projections unchanged. +func (r *MergeAdjacentProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + proj := old.GetExpr().ExprNode.(*plannercore.LogicalProjection) + childGroup := old.Children[0].Group + child := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) + if plannercore.ExprsHasSideEffects(child.Exprs) { + return nil, false, false, nil + } + + replace := make(map[string]*expression.Column) + for i, col := range childGroup.Prop.Schema.Columns { + if colOrigin, ok := child.Exprs[i].(*expression.Column); ok { + replace[string(col.HashCode(nil))] = colOrigin + } + } + + newProj := plannercore.LogicalProjection{Exprs: make([]expression.Expression, len(proj.Exprs))}.Init(proj.SCtx(), proj.SelectBlockOffset()) + newProj.SetSchema(old.GetExpr().Group.Prop.Schema) + for i, expr := range proj.Exprs { + newExpr := expr.Clone() + plannercore.ResolveExprAndReplace(newExpr, replace) + newProj.Exprs[i] = plannercore.ReplaceColumnOfExpr(newExpr, child, childGroup.Prop.Schema) + } + + newProjExpr := memo.NewGroupExpr(newProj) + newProjExpr.SetChildren(old.Children[0].GetExpr().Children[0]) + return []*memo.GroupExpr{newProjExpr}, true, false, nil +} + +// PushTopNDownOuterJoin pushes topN to outer join. +type PushTopNDownOuterJoin struct { + baseRule +} + +// NewRulePushTopNDownOuterJoin creates a new Transformation PushTopNDownOuterJoin. +// The pattern of this rule is: `TopN -> Join`. +func NewRulePushTopNDownOuterJoin() Transformation { + rule := &PushTopNDownOuterJoin{} + rule.pattern = memo.BuildPattern( + memo.OperandTopN, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandJoin, memo.EngineTiDBOnly), + ) + return rule +} + +// Match implements Transformation interface. +// Use appliedRuleSet in GroupExpr to avoid re-apply rules. +func (r *PushTopNDownOuterJoin) Match(expr *memo.ExprIter) bool { + if expr.GetExpr().HasAppliedRule(r) { + return false + } + join := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin) + switch join.JoinType { + case plannercore.LeftOuterJoin, plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin, plannercore.RightOuterJoin: + return true + default: + return false + } +} + +func pushTopNDownOuterJoinToChild(topN *plannercore.LogicalTopN, outerGroup *memo.Group) *memo.Group { + for _, by := range topN.ByItems { + cols := expression.ExtractColumns(by.Expr) + for _, col := range cols { + if !outerGroup.Prop.Schema.Contains(col) { + return outerGroup + } + } + } + + newTopN := plannercore.LogicalTopN{ + Count: topN.Count + topN.Offset, + ByItems: make([]*util.ByItems, len(topN.ByItems)), + }.Init(topN.SCtx(), topN.SelectBlockOffset()) + + for i := range topN.ByItems { + newTopN.ByItems[i] = topN.ByItems[i].Clone() + } + newTopNGroup := memo.NewGroupExpr(newTopN) + newTopNGroup.SetChildren(outerGroup) + newChild := memo.NewGroupWithSchema(newTopNGroup, outerGroup.Prop.Schema) + return newChild +} + +// OnTransform implements Transformation interface. +// This rule will transform `TopN->OuterJoin->(OuterChild, InnerChild)` to `TopN->OuterJoin->(TopN->OuterChild, InnerChild)` +func (r *PushTopNDownOuterJoin) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + topN := old.GetExpr().ExprNode.(*plannercore.LogicalTopN) + joinExpr := old.Children[0].GetExpr() + join := joinExpr.ExprNode.(*plannercore.LogicalJoin) + joinSchema := old.Children[0].Group.Prop.Schema + leftGroup := joinExpr.Children[0] + rightGroup := joinExpr.Children[1] + + switch join.JoinType { + case plannercore.LeftOuterJoin, plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin: + leftGroup = pushTopNDownOuterJoinToChild(topN, leftGroup) + case plannercore.RightOuterJoin: + rightGroup = pushTopNDownOuterJoinToChild(topN, rightGroup) + default: + return nil, false, false, nil + } + + newJoinExpr := memo.NewGroupExpr(join) + newJoinExpr.SetChildren(leftGroup, rightGroup) + newTopNExpr := memo.NewGroupExpr(topN) + newTopNExpr.SetChildren(memo.NewGroupWithSchema(newJoinExpr, joinSchema)) + newTopNExpr.AddAppliedRule(r) + return []*memo.GroupExpr{newTopNExpr}, true, false, nil +} + +// PushTopNDownProjection pushes TopN to Projection. +type PushTopNDownProjection struct { + baseRule +} + +// NewRulePushTopNDownProjection creates a new Transformation PushTopNDownProjection. +// The pattern of this rule is `TopN->Projection->X` to `Projection->TopN->X`. +func NewRulePushTopNDownProjection() Transformation { + rule := &PushTopNDownProjection{} + rule.pattern = memo.BuildPattern( + memo.OperandTopN, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandProjection, memo.EngineTiDBOnly), + ) + return rule +} + +// Match implements Transformation interface. +func (r *PushTopNDownProjection) Match(expr *memo.ExprIter) bool { + proj := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) + for _, expr := range proj.Exprs { + if expression.HasAssignSetVarFunc(expr) { + return false + } + } + return true +} + +// OnTransform implements Transformation interface. +// This rule tries to pushes the TopN through Projection. +func (r *PushTopNDownProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + topN := old.GetExpr().ExprNode.(*plannercore.LogicalTopN) + proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) + childGroup := old.Children[0].GetExpr().Children[0] + + newTopN := plannercore.LogicalTopN{ + Offset: topN.Offset, + Count: topN.Count, + }.Init(topN.SCtx(), topN.SelectBlockOffset()) + + newTopN.ByItems = make([]*util.ByItems, 0, len(topN.ByItems)) + for _, by := range topN.ByItems { + newTopN.ByItems = append(newTopN.ByItems, &util.ByItems{ + Expr: expression.ColumnSubstitute(by.Expr, old.Children[0].Group.Prop.Schema, proj.Exprs), + Desc: by.Desc, + }) + } + + // remove meaningless constant sort items. + for i := len(newTopN.ByItems) - 1; i >= 0; i-- { + switch newTopN.ByItems[i].Expr.(type) { + case *expression.Constant, *expression.CorrelatedColumn: + topN.ByItems = append(newTopN.ByItems[:i], newTopN.ByItems[i+1:]...) + } + } + projExpr := memo.NewGroupExpr(proj) + topNExpr := memo.NewGroupExpr(newTopN) + topNExpr.SetChildren(childGroup) + topNGroup := memo.NewGroupWithSchema(topNExpr, childGroup.Prop.Schema) + projExpr.SetChildren(topNGroup) + 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 +} + +// PushTopNDownTiKVSingleGather pushes the top-n down to child of TiKVSingleGather. +type PushTopNDownTiKVSingleGather struct { + baseRule +} + +// NewRulePushTopNDownTiKVSingleGather creates a new Transformation PushTopNDownTiKVSingleGather. +// The pattern of this rule is `TopN -> TiKVSingleGather`. +func NewRulePushTopNDownTiKVSingleGather() Transformation { + rule := &PushTopNDownTiKVSingleGather{} + rule.pattern = memo.BuildPattern( + memo.OperandTopN, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandTiKVSingleGather, memo.EngineTiDBOnly), + ) + return rule +} + +// Match implements Transformation interface. +// Use appliedRuleSet in GroupExpr to avoid re-apply rules. +func (r *PushTopNDownTiKVSingleGather) Match(expr *memo.ExprIter) bool { + return !expr.GetExpr().HasAppliedRule(r) +} + +// OnTransform implements Transformation interface. +// It transforms `TopN -> TiKVSingleGather` to `TopN(Final) -> TiKVSingleGather -> TopN(Partial)`. +func (r *PushTopNDownTiKVSingleGather) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + topN := old.GetExpr().ExprNode.(*plannercore.LogicalTopN) + topNSchema := old.Children[0].Group.Prop.Schema + gather := old.Children[0].GetExpr().ExprNode.(*plannercore.TiKVSingleGather) + childGroup := old.Children[0].GetExpr().Children[0] + + particalTopN := plannercore.LogicalTopN{ + ByItems: topN.ByItems, + Count: topN.Count + topN.Offset, + }.Init(topN.SCtx(), topN.SelectBlockOffset()) + partialTopNExpr := memo.NewGroupExpr(particalTopN) + partialTopNExpr.SetChildren(childGroup) + partialTopNGroup := memo.NewGroupWithSchema(partialTopNExpr, topNSchema).SetEngineType(childGroup.EngineType) + + gatherExpr := memo.NewGroupExpr(gather) + gatherExpr.SetChildren(partialTopNGroup) + gatherGroup := memo.NewGroupWithSchema(gatherExpr, topNSchema) + + finalTopNExpr := memo.NewGroupExpr(topN) + finalTopNExpr.SetChildren(gatherGroup) + finalTopNExpr.AddAppliedRule(r) + return []*memo.GroupExpr{finalTopNExpr}, true, false, nil +} + +// MergeAdjacentTopN merge adjacent TopN. +type MergeAdjacentTopN struct { + baseRule +} + +// NewRuleMergeAdjacentTopN creates a new Transformation MergeAdjacentTopN. +// The pattern of this rule is `TopN->TopN->X`. +func NewRuleMergeAdjacentTopN() Transformation { + rule := &MergeAdjacentTopN{} + rule.pattern = memo.BuildPattern( + memo.OperandTopN, + memo.EngineAll, + memo.NewPattern(memo.OperandTopN, memo.EngineAll), + ) + return rule +} + +// Match implements Transformation interface. +func (r *MergeAdjacentTopN) Match(expr *memo.ExprIter) bool { + topN := expr.GetExpr().ExprNode.(*plannercore.LogicalTopN) + child := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalTopN) + + // We can use this rule when the sort columns of parent TopN is a prefix of child TopN. + if len(child.ByItems) < len(topN.ByItems) { + return false + } + for i := 0; i < len(topN.ByItems); i++ { + if !topN.ByItems[i].Equal(topN.SCtx(), child.ByItems[i]) { + return false + } + } + return true +} + +// OnTransform implements Transformation interface. +// This rule tries to merge adjacent TopN. +func (r *MergeAdjacentTopN) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + topN := old.GetExpr().ExprNode.(*plannercore.LogicalTopN) + child := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalTopN) + childGroups := old.Children[0].GetExpr().Children + + if child.Count <= topN.Offset { + tableDual := plannercore.LogicalTableDual{RowCount: 0}.Init(child.SCtx(), child.SelectBlockOffset()) + tableDual.SetSchema(old.GetExpr().Schema()) + tableDualExpr := memo.NewGroupExpr(tableDual) + return []*memo.GroupExpr{tableDualExpr}, true, true, nil + } + + offset := child.Offset + topN.Offset + count := uint64(math.Min(float64(child.Count-topN.Offset), float64(topN.Count))) + newTopN := plannercore.LogicalTopN{ + Count: count, + Offset: offset, + ByItems: child.ByItems, + }.Init(child.SCtx(), child.SelectBlockOffset()) + newTopNExpr := memo.NewGroupExpr(newTopN) + newTopNExpr.SetChildren(childGroups...) + 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. +type MergeAggregationProjection struct { + baseRule +} + +// NewRuleMergeAggregationProjection creates a new Transformation MergeAggregationProjection. +// The pattern of this rule is: `Aggregation -> Projection`. +func NewRuleMergeAggregationProjection() Transformation { + rule := &MergeAggregationProjection{} + rule.pattern = memo.BuildPattern( + memo.OperandAggregation, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandProjection, memo.EngineTiDBOnly), + ) + return rule +} + +// Match implements Transformation interface. +func (r *MergeAggregationProjection) Match(old *memo.ExprIter) bool { + proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) + if plannercore.ExprsHasSideEffects(proj.Exprs) { + return false + } + return true +} + +// OnTransform implements Transformation interface. +// It will transform `Aggregation->Projection->X` to `Aggregation->X`. +func (r *MergeAggregationProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + oldAgg := old.GetExpr().ExprNode.(*plannercore.LogicalAggregation) + proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) + projSchema := old.Children[0].GetExpr().Schema() + + groupByItems := make([]expression.Expression, len(oldAgg.GroupByItems)) + for i, item := range oldAgg.GroupByItems { + groupByItems[i] = expression.ColumnSubstitute(item, projSchema, proj.Exprs) + } + + aggFuncs := make([]*aggregation.AggFuncDesc, len(oldAgg.AggFuncs)) + for i, aggFunc := range oldAgg.AggFuncs { + aggFuncs[i] = aggFunc.Clone() + newArgs := make([]expression.Expression, len(aggFunc.Args)) + for j, arg := range aggFunc.Args { + newArgs[j] = expression.ColumnSubstitute(arg, projSchema, proj.Exprs) + } + aggFuncs[i].Args = newArgs + } + + newAgg := plannercore.LogicalAggregation{ + GroupByItems: groupByItems, + AggFuncs: aggFuncs, + }.Init(oldAgg.SCtx(), oldAgg.SelectBlockOffset()) + + newAggExpr := memo.NewGroupExpr(newAgg) + newAggExpr.SetChildren(old.Children[0].GetExpr().Children...) + return []*memo.GroupExpr{newAggExpr}, false, 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 + } + + // If there is only one aggFunc, we don't need to guarantee that the child of it is a data + // source, or whether the sort can be eliminated. This transformation won't be worse than previous. + // Make sure that the aggFunc are Max or Min. + // TODO: If there have only one Max or Min aggFunc and the other aggFuncs are FirstRow() can also use this rule. Waiting for the not null prop is maintained. + if len(agg.AggFuncs) != 1 { + return false + } + if agg.AggFuncs[0].Name != ast.AggFuncMax && agg.AggFuncs[0].Name != ast.AggFuncMin { + 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() + f := agg.AggFuncs[0] + + // 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[0])) > 0 { + // If it can be NULL, we need to filter NULL out first. + if !mysql.HasNotNullFlag(f.Args[0].GetType().Flag) { + sel := plannercore.LogicalSelection{}.Init(ctx, agg.SelectBlockOffset()) + isNullFunc := expression.NewFunctionInternal(ctx, ast.IsNull, types.NewFieldType(mysql.TypeTiny), f.Args[0]) + 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 []*util.ByItems + byItems = append(byItems, &util.ByItems{ + Expr: f.Args[0], + 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 +} + +// NewRuleMergeAdjacentSelection creates a new Transformation MergeAdjacentSelection. +// The pattern of this rule is `Selection->Selection->X`. +func NewRuleMergeAdjacentSelection() Transformation { + rule := &MergeAdjacentSelection{} + rule.pattern = memo.BuildPattern( + memo.OperandSelection, + memo.EngineAll, + memo.NewPattern(memo.OperandSelection, memo.EngineAll), + ) + return rule +} + +// OnTransform implements Transformation interface. +// This rule tries to merge adjacent selection, with no simplification. +func (r *MergeAdjacentSelection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) + child := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalSelection) + childGroups := old.Children[0].GetExpr().Children + + conditions := make([]expression.Expression, 0, len(sel.Conditions)+len(child.Conditions)) + conditions = append(conditions, sel.Conditions...) + conditions = append(conditions, child.Conditions...) + newSel := plannercore.LogicalSelection{Conditions: conditions}.Init(sel.SCtx(), sel.SelectBlockOffset()) + newSelExpr := memo.NewGroupExpr(newSel) + newSelExpr.SetChildren(childGroups...) + return []*memo.GroupExpr{newSelExpr}, true, false, nil +} + +// MergeAdjacentLimit merge the adjacent limit. +type MergeAdjacentLimit struct { + baseRule +} + +// NewRuleMergeAdjacentLimit creates a new Transformation MergeAdjacentLimit. +// The pattern of this rule is `Limit->Limit->X`. +func NewRuleMergeAdjacentLimit() Transformation { + rule := &MergeAdjacentLimit{} + rule.pattern = memo.BuildPattern( + memo.OperandLimit, + memo.EngineAll, + memo.NewPattern(memo.OperandLimit, memo.EngineAll), + ) + return rule +} + +// OnTransform implements Transformation interface. +// This rule tries to merge adjacent limit. +func (r *MergeAdjacentLimit) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + limit := old.GetExpr().ExprNode.(*plannercore.LogicalLimit) + child := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalLimit) + childGroups := old.Children[0].GetExpr().Children + + if child.Count <= limit.Offset { + tableDual := plannercore.LogicalTableDual{RowCount: 0}.Init(child.SCtx(), child.SelectBlockOffset()) + tableDual.SetSchema(old.GetExpr().Schema()) + tableDualExpr := memo.NewGroupExpr(tableDual) + return []*memo.GroupExpr{tableDualExpr}, true, true, nil + } + + offset := child.Offset + limit.Offset + count := uint64(math.Min(float64(child.Count-limit.Offset), float64(limit.Count))) + newLimit := plannercore.LogicalLimit{ + Offset: offset, + Count: count, + }.Init(limit.SCtx(), limit.SelectBlockOffset()) + newLimitExpr := memo.NewGroupExpr(newLimit) + newLimitExpr.SetChildren(childGroups...) + return []*memo.GroupExpr{newLimitExpr}, true, false, nil +} + +// TransformLimitToTableDual convert limit to TableDual. +type TransformLimitToTableDual struct { + baseRule +} + +// NewRuleTransformLimitToTableDual creates a new Transformation TransformLimitToTableDual. +// The pattern of this rule is `Limit->X`. +func NewRuleTransformLimitToTableDual() Transformation { + rule := &TransformLimitToTableDual{} + rule.pattern = memo.BuildPattern( + memo.OperandLimit, + memo.EngineAll, + ) + return rule +} + +// Match implements Transformation interface. +func (r *TransformLimitToTableDual) Match(expr *memo.ExprIter) bool { + limit := expr.GetExpr().ExprNode.(*plannercore.LogicalLimit) + return 0 == limit.Count +} + +// OnTransform implements Transformation interface. +// This rule tries to convert limit to tableDual. +func (r *TransformLimitToTableDual) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + limit := old.GetExpr().ExprNode.(*plannercore.LogicalLimit) + tableDual := plannercore.LogicalTableDual{RowCount: 0}.Init(limit.SCtx(), limit.SelectBlockOffset()) + tableDual.SetSchema(old.GetExpr().Schema()) + tableDualExpr := memo.NewGroupExpr(tableDual) + return []*memo.GroupExpr{tableDualExpr}, true, true, nil +} + +// PushLimitDownOuterJoin pushes Limit through Join. +type PushLimitDownOuterJoin struct { + baseRule +} + +// NewRulePushLimitDownOuterJoin creates a new Transformation PushLimitDownOuterJoin. +// The pattern of this rule is `Limit -> Join`. +func NewRulePushLimitDownOuterJoin() Transformation { + rule := &PushLimitDownOuterJoin{} + rule.pattern = memo.BuildPattern( + memo.OperandLimit, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandJoin, memo.EngineTiDBOnly), + ) + return rule +} + +// Match implements Transformation interface. +func (r *PushLimitDownOuterJoin) Match(expr *memo.ExprIter) bool { + if expr.GetExpr().HasAppliedRule(r) { + return false + } + join := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin) + return join.JoinType.IsOuterJoin() +} + +// OnTransform implements Transformation interface. +// This rule tries to pushes the Limit through outer Join. +func (r *PushLimitDownOuterJoin) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + limit := old.GetExpr().ExprNode.(*plannercore.LogicalLimit) + join := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin) + joinSchema := old.Children[0].Group.Prop.Schema + leftGroup := old.Children[0].GetExpr().Children[0] + rightGroup := old.Children[0].GetExpr().Children[1] + + switch join.JoinType { + case plannercore.LeftOuterJoin, plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin: + leftGroup = r.pushLimitDownOuterJoinToChild(limit, leftGroup) + case plannercore.RightOuterJoin: + rightGroup = r.pushLimitDownOuterJoinToChild(limit, rightGroup) + default: + return nil, false, false, nil + } + + newJoinExpr := memo.NewGroupExpr(join) + newJoinExpr.SetChildren(leftGroup, rightGroup) + newLimitExpr := memo.NewGroupExpr(limit) + newLimitExpr.SetChildren(memo.NewGroupWithSchema(newJoinExpr, joinSchema)) + newLimitExpr.AddAppliedRule(r) + return []*memo.GroupExpr{newLimitExpr}, true, false, nil +} + +func (r *PushLimitDownOuterJoin) pushLimitDownOuterJoinToChild(limit *plannercore.LogicalLimit, outerGroup *memo.Group) *memo.Group { + newLimit := plannercore.LogicalLimit{ + Count: limit.Count + limit.Offset, + }.Init(limit.SCtx(), limit.SelectBlockOffset()) + newLimitGroup := memo.NewGroupExpr(newLimit) + newLimitGroup.SetChildren(outerGroup) + return memo.NewGroupWithSchema(newLimitGroup, outerGroup.Prop.Schema) +} + +// PushLimitDownTiKVSingleGather pushes the limit down to child of TiKVSingleGather. +type PushLimitDownTiKVSingleGather struct { + baseRule +} + +// NewRulePushLimitDownTiKVSingleGather creates a new Transformation PushLimitDownTiKVSingleGather. +// The pattern of this rule is `Limit -> TiKVSingleGather`. +func NewRulePushLimitDownTiKVSingleGather() Transformation { + rule := &PushLimitDownTiKVSingleGather{} + rule.pattern = memo.BuildPattern( + memo.OperandLimit, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandTiKVSingleGather, memo.EngineTiDBOnly), + ) + return rule +} + +// Match implements Transformation interface. +// Use appliedRuleSet in GroupExpr to avoid re-apply rules. +func (r *PushLimitDownTiKVSingleGather) Match(expr *memo.ExprIter) bool { + return !expr.GetExpr().HasAppliedRule(r) +} + +// OnTransform implements Transformation interface. +// It transforms `Limit -> TiKVSingleGather` to `Limit(Final) -> TiKVSingleGather -> Limit(Partial)`. +func (r *PushLimitDownTiKVSingleGather) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + limit := old.GetExpr().ExprNode.(*plannercore.LogicalLimit) + limitSchema := old.Children[0].Group.Prop.Schema + gather := old.Children[0].GetExpr().ExprNode.(*plannercore.TiKVSingleGather) + childGroup := old.Children[0].GetExpr().Children[0] + + particalLimit := plannercore.LogicalLimit{ + Count: limit.Count + limit.Offset, + }.Init(limit.SCtx(), limit.SelectBlockOffset()) + partialLimitExpr := memo.NewGroupExpr(particalLimit) + partialLimitExpr.SetChildren(childGroup) + partialLimitGroup := memo.NewGroupWithSchema(partialLimitExpr, limitSchema).SetEngineType(childGroup.EngineType) + + gatherExpr := memo.NewGroupExpr(gather) + gatherExpr.SetChildren(partialLimitGroup) + gatherGroup := memo.NewGroupWithSchema(gatherExpr, limitSchema) + + finalLimitExpr := memo.NewGroupExpr(limit) + finalLimitExpr.SetChildren(gatherGroup) + finalLimitExpr.AddAppliedRule(r) + return []*memo.GroupExpr{finalLimitExpr}, true, false, nil +} + +type outerJoinEliminator struct { +} + +func (*outerJoinEliminator) prepareForEliminateOuterJoin(joinExpr *memo.GroupExpr) (ok bool, innerChildIdx int, outerGroup *memo.Group, innerGroup *memo.Group, outerUniqueIDs set.Int64Set) { + join := joinExpr.ExprNode.(*plannercore.LogicalJoin) + + switch join.JoinType { + case plannercore.LeftOuterJoin: + innerChildIdx = 1 + case plannercore.RightOuterJoin: + innerChildIdx = 0 + default: + ok = false + return + } + outerGroup = joinExpr.Children[1^innerChildIdx] + innerGroup = joinExpr.Children[innerChildIdx] + + outerUniqueIDs = set.NewInt64Set() + for _, outerCol := range outerGroup.Prop.Schema.Columns { + outerUniqueIDs.Insert(outerCol.UniqueID) + } + + ok = true + return +} + +// check whether one of unique keys sets is contained by inner join keys. +func (*outerJoinEliminator) isInnerJoinKeysContainUniqueKey(innerGroup *memo.Group, joinKeys *expression.Schema) (bool, error) { + // builds UniqueKey info of innerGroup. + innerGroup.BuildKeyInfo() + for _, keyInfo := range innerGroup.Prop.Schema.Keys { + joinKeysContainKeyInfo := true + for _, col := range keyInfo { + if !joinKeys.Contains(col) { + joinKeysContainKeyInfo = false + break + } + } + if joinKeysContainKeyInfo { + return true, nil + } + } + return false, nil +} + +// EliminateOuterJoinBelowAggregation eliminate the outer join which below aggregation. +type EliminateOuterJoinBelowAggregation struct { + baseRule + outerJoinEliminator +} + +// NewRuleEliminateOuterJoinBelowAggregation creates a new Transformation EliminateOuterJoinBelowAggregation. +// The pattern of this rule is `Aggregation->Join->X`. +func NewRuleEliminateOuterJoinBelowAggregation() Transformation { + rule := &EliminateOuterJoinBelowAggregation{} + rule.pattern = memo.BuildPattern( + memo.OperandAggregation, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandJoin, memo.EngineTiDBOnly), + ) + return rule +} + +// Match implements Transformation interface. +func (r *EliminateOuterJoinBelowAggregation) Match(expr *memo.ExprIter) bool { + joinType := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin).JoinType + return joinType == plannercore.LeftOuterJoin || joinType == plannercore.RightOuterJoin +} + +// OnTransform implements Transformation interface. +// This rule tries to eliminate outer join which below aggregation. +func (r *EliminateOuterJoinBelowAggregation) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + agg := old.GetExpr().ExprNode.(*plannercore.LogicalAggregation) + joinExpr := old.Children[0].GetExpr() + join := joinExpr.ExprNode.(*plannercore.LogicalJoin) + + ok, innerChildIdx, outerGroup, innerGroup, outerUniqueIDs := r.prepareForEliminateOuterJoin(joinExpr) + if !ok { + return nil, false, false, nil + } + + // only when agg only use the columns from outer table can eliminate outer join. + if !plannercore.IsColsAllFromOuterTable(agg.GetUsedCols(), outerUniqueIDs) { + return nil, false, false, nil + } + // outer join elimination with duplicate agnostic aggregate functions. + _, aggCols := plannercore.GetDupAgnosticAggCols(agg, nil) + if len(aggCols) > 0 { + newAggExpr := memo.NewGroupExpr(agg) + newAggExpr.SetChildren(outerGroup) + return []*memo.GroupExpr{newAggExpr}, true, false, nil + } + // outer join elimination without duplicate agnostic aggregate functions. + innerJoinKeys := join.ExtractJoinKeys(innerChildIdx) + contain, err := r.isInnerJoinKeysContainUniqueKey(innerGroup, innerJoinKeys) + if err != nil { + return nil, false, false, err + } + if contain { + newAggExpr := memo.NewGroupExpr(agg) + newAggExpr.SetChildren(outerGroup) + return []*memo.GroupExpr{newAggExpr}, true, false, nil + } + + return nil, false, false, nil +} + +// EliminateOuterJoinBelowProjection eliminate the outer join which below projection. +type EliminateOuterJoinBelowProjection struct { + baseRule + outerJoinEliminator +} + +// NewRuleEliminateOuterJoinBelowProjection creates a new Transformation EliminateOuterJoinBelowProjection. +// The pattern of this rule is `Projection->Join->X`. +func NewRuleEliminateOuterJoinBelowProjection() Transformation { + rule := &EliminateOuterJoinBelowProjection{} + rule.pattern = memo.BuildPattern( + memo.OperandProjection, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandJoin, memo.EngineTiDBOnly), + ) + return rule +} + +// Match implements Transformation interface. +func (r *EliminateOuterJoinBelowProjection) Match(expr *memo.ExprIter) bool { + joinType := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin).JoinType + return joinType == plannercore.LeftOuterJoin || joinType == plannercore.RightOuterJoin +} + +// OnTransform implements Transformation interface. +// This rule tries to eliminate outer join which below projection. +func (r *EliminateOuterJoinBelowProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + proj := old.GetExpr().ExprNode.(*plannercore.LogicalProjection) + joinExpr := old.Children[0].GetExpr() + join := joinExpr.ExprNode.(*plannercore.LogicalJoin) + + ok, innerChildIdx, outerGroup, innerGroup, outerUniqueIDs := r.prepareForEliminateOuterJoin(joinExpr) + if !ok { + return nil, false, false, nil + } + + // only when proj only use the columns from outer table can eliminate outer join. + if !plannercore.IsColsAllFromOuterTable(proj.GetUsedCols(), outerUniqueIDs) { + return nil, false, false, nil + } + + innerJoinKeys := join.ExtractJoinKeys(innerChildIdx) + contain, err := r.isInnerJoinKeysContainUniqueKey(innerGroup, innerJoinKeys) + if err != nil { + return nil, false, false, err + } + if contain { + newProjExpr := memo.NewGroupExpr(proj) + newProjExpr.SetChildren(outerGroup) + return []*memo.GroupExpr{newProjExpr}, true, false, nil + } + + return nil, false, false, nil +} + +// TransformAggregateCaseToSelection convert Agg(case when) to Agg->Selection. +type TransformAggregateCaseToSelection struct { + baseRule +} + +// NewRuleTransformAggregateCaseToSelection creates a new Transformation TransformAggregateCaseToSelection. +// The pattern of this rule is `Agg->X`. +func NewRuleTransformAggregateCaseToSelection() Transformation { + rule := &TransformAggregateCaseToSelection{} + rule.pattern = memo.BuildPattern( + memo.OperandAggregation, + memo.EngineTiDBOnly, + ) + return rule +} + +// Match implements Transformation interface. +func (r *TransformAggregateCaseToSelection) Match(expr *memo.ExprIter) bool { + agg := expr.GetExpr().ExprNode.(*plannercore.LogicalAggregation) + return agg.IsCompleteModeAgg() && len(agg.GroupByItems) == 0 && len(agg.AggFuncs) == 1 && len(agg.AggFuncs[0].Args) == 1 && r.isTwoOrThreeArgCase(agg.AggFuncs[0].Args[0]) +} + +// OnTransform implements Transformation interface. +// This rule tries to convert Agg(case when) to Agg->Selection. +func (r *TransformAggregateCaseToSelection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + agg := old.GetExpr().ExprNode.(*plannercore.LogicalAggregation) + + ok, newConditions, newAggFuncs := r.transform(agg) + if !ok { + return nil, false, false, nil + } + + newSel := plannercore.LogicalSelection{Conditions: newConditions}.Init(agg.SCtx(), agg.SelectBlockOffset()) + newSelExpr := memo.NewGroupExpr(newSel) + newSelExpr.SetChildren(old.GetExpr().Children...) + newSelGroup := memo.NewGroupWithSchema(newSelExpr, old.GetExpr().Children[0].Prop.Schema) + + newAgg := plannercore.LogicalAggregation{ + AggFuncs: newAggFuncs, + GroupByItems: agg.GroupByItems, + }.Init(agg.SCtx(), agg.SelectBlockOffset()) + newAgg.CopyAggHints(agg) + newAggExpr := memo.NewGroupExpr(newAgg) + newAggExpr.SetChildren(newSelGroup) + return []*memo.GroupExpr{newAggExpr}, true, false, nil +} + +func (r *TransformAggregateCaseToSelection) transform(agg *plannercore.LogicalAggregation) (ok bool, newConditions []expression.Expression, newAggFuncs []*aggregation.AggFuncDesc) { + aggFuncDesc := agg.AggFuncs[0] + aggFuncName := aggFuncDesc.Name + ctx := agg.SCtx() + + caseFunc := aggFuncDesc.Args[0].(*expression.ScalarFunction) + conditionFromCase := caseFunc.GetArgs()[0] + caseArgs := caseFunc.GetArgs() + caseArgsNum := len(caseArgs) + + // `case when a>0 then null else a end` should be converted to `case when !(a>0) then a else null end`. + var nullFlip = caseArgsNum == 3 && caseArgs[1].Equal(ctx, expression.NewNull()) && !caseArgs[2].Equal(ctx, expression.NewNull()) + // `case when a>0 then 0 else a end` should be converted to `case when !(a>0) then a else 0 end`. + var zeroFlip = !nullFlip && caseArgsNum == 3 && caseArgs[1].Equal(ctx, expression.NewZero()) + + var outputIdx int + if nullFlip || zeroFlip { + outputIdx = 2 + newConditions = []expression.Expression{expression.NewFunctionInternal(ctx, ast.UnaryNot, types.NewFieldType(mysql.TypeTiny), conditionFromCase)} + } else { + outputIdx = 1 + newConditions = expression.SplitCNFItems(conditionFromCase) + } + + if aggFuncDesc.HasDistinct { + // Just one style supported: + // COUNT(DISTINCT CASE WHEN x = 'foo' THEN y END) + // => + // newAggFuncDesc: COUNT(DISTINCT y), newCondition: x = 'foo' + + if aggFuncName == ast.AggFuncCount && r.isOnlyOneNotNull(ctx, caseArgs, caseArgsNum, outputIdx) { + newAggFuncDesc := aggFuncDesc.Clone() + newAggFuncDesc.Args = []expression.Expression{caseArgs[outputIdx]} + return true, newConditions, []*aggregation.AggFuncDesc{newAggFuncDesc} + } + return false, nil, nil + } + + // Two styles supported: + // + // A1: AGG(CASE WHEN x = 'foo' THEN cnt END) + // => newAggFuncDesc: AGG(cnt), newCondition: x = 'foo' + // A2: SUM(CASE WHEN x = 'foo' THEN cnt ELSE 0 END) + // => newAggFuncDesc: SUM(cnt), newCondition: x = 'foo' + + switch { + case r.allowsSelection(aggFuncName) && (caseArgsNum == 2 || caseArgs[3-outputIdx].Equal(ctx, expression.NewNull())), // Case A1 + aggFuncName == ast.AggFuncSum && caseArgsNum == 3 && caseArgs[3-outputIdx].Equal(ctx, expression.NewZero()): // Case A2 + newAggFuncDesc := aggFuncDesc.Clone() + newAggFuncDesc.Args = []expression.Expression{caseArgs[outputIdx]} + return true, newConditions, []*aggregation.AggFuncDesc{newAggFuncDesc} + default: + return false, nil, nil + } +} + +func (r *TransformAggregateCaseToSelection) allowsSelection(aggFuncName string) bool { + return aggFuncName != ast.AggFuncFirstRow +} + +func (r *TransformAggregateCaseToSelection) isOnlyOneNotNull(ctx sessionctx.Context, args []expression.Expression, argsNum int, outputIdx int) bool { + return !args[outputIdx].Equal(ctx, expression.NewNull()) && (argsNum == 2 || args[3-outputIdx].Equal(ctx, expression.NewNull())) +} + +// TransformAggregateCaseToSelection only support `case when cond then var end` and `case when cond then var1 else var2 end`. +func (r *TransformAggregateCaseToSelection) isTwoOrThreeArgCase(expr expression.Expression) bool { + scalarFunc, ok := expr.(*expression.ScalarFunction) + if !ok { + return false + } + return scalarFunc.FuncName.L == ast.Case && (len(scalarFunc.GetArgs()) == 2 || len(scalarFunc.GetArgs()) == 3) +} + +// TransformAggToProj convert Agg to Proj. +type TransformAggToProj struct { + baseRule +} + +// NewRuleTransformAggToProj creates a new Transformation TransformAggToProj. +// The pattern of this rule is `Agg`. +func NewRuleTransformAggToProj() Transformation { + rule := &TransformAggToProj{} + rule.pattern = memo.BuildPattern( + memo.OperandAggregation, + memo.EngineTiDBOnly, + ) + return rule +} + +// Match implements Transformation interface. +func (r *TransformAggToProj) Match(expr *memo.ExprIter) bool { + agg := expr.GetExpr().ExprNode.(*plannercore.LogicalAggregation) + + if !agg.IsCompleteModeAgg() { + return false + } + + for _, af := range agg.AggFuncs { + // TODO(issue #9968): same as rule_aggregation_elimination.go -> tryToEliminateAggregation. + // waiting for (issue #14616): `nullable` information. + if af.Name == ast.AggFuncGroupConcat { + return false + } + } + + childGroup := expr.GetExpr().Children[0] + childGroup.BuildKeyInfo() + schemaByGroupby := expression.NewSchema(agg.GetGroupByCols()...) + for _, key := range childGroup.Prop.Schema.Keys { + if schemaByGroupby.ColumnsIndices(key) != nil { + return true + } + } + + return false +} + +// OnTransform implements Transformation interface. +// This rule tries to convert agg to proj. +func (r *TransformAggToProj) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + agg := old.GetExpr().ExprNode.(*plannercore.LogicalAggregation) + if ok, proj := plannercore.ConvertAggToProj(agg, old.GetExpr().Schema()); ok { + newProjExpr := memo.NewGroupExpr(proj) + newProjExpr.SetChildren(old.GetExpr().Children...) + return []*memo.GroupExpr{newProjExpr}, true, false, nil + } + + return nil, false, false, nil +} + +// InjectProjectionBelowTopN injects two Projections below and upon TopN if TopN's ByItems +// contain ScalarFunctions. +type InjectProjectionBelowTopN struct { + baseRule +} + +// NewRuleInjectProjectionBelowTopN creates a new Transformation InjectProjectionBelowTopN. +// It will extract the ScalarFunctions of `ByItems` into a Projection and injects it below TopN. +// When a Projection is injected as the child of TopN, we need to add another Projection upon +// TopN to prune the extra Columns. +// The reason why we need this rule is that, TopNExecutor in TiDB does not support ScalarFunction +// as `ByItem`. So we have to use a Projection to calculate the ScalarFunctions in advance. +// The pattern of this rule is: a single TopN +func NewRuleInjectProjectionBelowTopN() Transformation { + rule := &InjectProjectionBelowTopN{} + rule.pattern = memo.BuildPattern( + memo.OperandTopN, + memo.EngineTiDBOnly, + ) + return rule +} + +// Match implements Transformation interface. +func (r *InjectProjectionBelowTopN) Match(expr *memo.ExprIter) bool { + topN := expr.GetExpr().ExprNode.(*plannercore.LogicalTopN) + for _, item := range topN.ByItems { + if _, ok := item.Expr.(*expression.ScalarFunction); ok { + return true + } + } + return false +} + +// OnTransform implements Transformation interface. +// It will convert `TopN -> X` to `Projection -> TopN -> Projection -> X`. +func (r *InjectProjectionBelowTopN) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + topN := old.GetExpr().ExprNode.(*plannercore.LogicalTopN) + oldTopNSchema := old.GetExpr().Schema() + + // Construct top Projection. + topProjExprs := make([]expression.Expression, oldTopNSchema.Len()) + for i := range oldTopNSchema.Columns { + topProjExprs[i] = oldTopNSchema.Columns[i] + } + topProj := plannercore.LogicalProjection{ + Exprs: topProjExprs, + }.Init(topN.SCtx(), topN.SelectBlockOffset()) + topProj.SetSchema(oldTopNSchema) + + // Construct bottom Projection. + bottomProjExprs := make([]expression.Expression, 0, oldTopNSchema.Len()+len(topN.ByItems)) + bottomProjSchema := make([]*expression.Column, 0, oldTopNSchema.Len()+len(topN.ByItems)) + for _, col := range oldTopNSchema.Columns { + bottomProjExprs = append(bottomProjExprs, col) + bottomProjSchema = append(bottomProjSchema, col) + } + newByItems := make([]*util.ByItems, 0, len(topN.ByItems)) + for _, item := range topN.ByItems { + itemExpr := item.Expr + if _, isScalarFunc := itemExpr.(*expression.ScalarFunction); !isScalarFunc { + newByItems = append(newByItems, item) + continue + } + bottomProjExprs = append(bottomProjExprs, itemExpr) + newCol := &expression.Column{ + UniqueID: topN.SCtx().GetSessionVars().AllocPlanColumnID(), + RetType: itemExpr.GetType(), + } + bottomProjSchema = append(bottomProjSchema, newCol) + newByItems = append(newByItems, &util.ByItems{Expr: newCol, Desc: item.Desc}) + } + bottomProj := plannercore.LogicalProjection{ + Exprs: bottomProjExprs, + }.Init(topN.SCtx(), topN.SelectBlockOffset()) + newSchema := expression.NewSchema(bottomProjSchema...) + bottomProj.SetSchema(newSchema) + + newTopN := plannercore.LogicalTopN{ + ByItems: newByItems, + Offset: topN.Offset, + Count: topN.Count, + }.Init(topN.SCtx(), topN.SelectBlockOffset()) + + // Construct GroupExpr, Group (TopProj -> TopN -> BottomProj -> Child) + bottomProjGroupExpr := memo.NewGroupExpr(bottomProj) + bottomProjGroupExpr.SetChildren(old.GetExpr().Children[0]) + bottomProjGroup := memo.NewGroupWithSchema(bottomProjGroupExpr, newSchema) + + topNGroupExpr := memo.NewGroupExpr(newTopN) + topNGroupExpr.SetChildren(bottomProjGroup) + topNGroup := memo.NewGroupWithSchema(topNGroupExpr, newSchema) + + topProjGroupExpr := memo.NewGroupExpr(topProj) + topProjGroupExpr.SetChildren(topNGroup) + return []*memo.GroupExpr{topProjGroupExpr}, true, false, nil +} + +// TransformApplyToJoin transforms a LogicalApply to LogicalJoin if it's +// inner children has no correlated columns from it's outer schema. +type TransformApplyToJoin struct { + baseRule +} + +// NewRuleTransformApplyToJoin creates a new Transformation TransformApplyToJoin. +// The pattern of this rule is: `Apply -> (X, Y)`. +func NewRuleTransformApplyToJoin() Transformation { + rule := &TransformApplyToJoin{} + rule.pattern = memo.NewPattern(memo.OperandApply, memo.EngineTiDBOnly) + return rule +} + +// OnTransform implements Transformation interface. +func (r *TransformApplyToJoin) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + apply := old.GetExpr().ExprNode.(*plannercore.LogicalApply) + groupExpr := old.GetExpr() + // It's safe to use the old apply instead of creating a new LogicalApply here, + // Because apply.CorCols will only be used and updated by this rule during Transformation. + apply.CorCols = r.extractCorColumnsBySchema(groupExpr.Children[1], groupExpr.Children[0].Prop.Schema) + if len(apply.CorCols) != 0 { + return nil, false, false, nil + } + + join := apply.LogicalJoin.Shallow() + joinGroupExpr := memo.NewGroupExpr(join) + joinGroupExpr.SetChildren(groupExpr.Children...) + return []*memo.GroupExpr{joinGroupExpr}, true, false, nil +} + +func (r *TransformApplyToJoin) extractCorColumnsBySchema(innerGroup *memo.Group, outerSchema *expression.Schema) []*expression.CorrelatedColumn { + corCols := r.extractCorColumnsFromGroup(innerGroup) + return plannercore.ExtractCorColumnsBySchema(corCols, outerSchema) +} + +func (r *TransformApplyToJoin) extractCorColumnsFromGroup(g *memo.Group) []*expression.CorrelatedColumn { + corCols := make([]*expression.CorrelatedColumn, 0) + for elem := g.Equivalents.Front(); elem != nil; elem = elem.Next() { + expr := elem.Value.(*memo.GroupExpr) + corCols = append(corCols, expr.ExprNode.ExtractCorrelatedCols()...) + for _, child := range expr.Children { + corCols = append(corCols, r.extractCorColumnsFromGroup(child)...) + } + } + // We may have duplicate CorrelatedColumns here, but it won't influence + // the logic of the transformation. Apply.CorCols will be deduplicated in + // `ResolveIndices`. + return corCols +} + +// PullSelectionUpApply pulls up the inner-side Selection into Apply as +// its join condition. +type PullSelectionUpApply struct { + baseRule +} + +// NewRulePullSelectionUpApply creates a new Transformation PullSelectionUpApply. +// The pattern of this rule is: `Apply -> (Any, Selection)`. +func NewRulePullSelectionUpApply() Transformation { + rule := &PullSelectionUpApply{} + rule.pattern = memo.BuildPattern( + memo.OperandApply, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandAny, memo.EngineTiDBOnly), // outer child + memo.NewPattern(memo.OperandSelection, memo.EngineTiDBOnly), // inner child + ) + return rule +} + +// OnTransform implements Transformation interface. +// This rule tries to pull up the inner side Selection, and add these conditions +// to Join condition inside the Apply. +func (r *PullSelectionUpApply) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + apply := old.GetExpr().ExprNode.(*plannercore.LogicalApply) + outerChildGroup := old.Children[0].Group + innerChildGroup := old.Children[1].Group + sel := old.Children[1].GetExpr().ExprNode.(*plannercore.LogicalSelection) + newConds := make([]expression.Expression, 0, len(sel.Conditions)) + for _, cond := range sel.Conditions { + newConds = append(newConds, cond.Clone().Decorrelate(outerChildGroup.Prop.Schema)) + } + newApply := plannercore.LogicalApply{ + LogicalJoin: *(apply.LogicalJoin.Shallow()), + CorCols: apply.CorCols, + }.Init(apply.SCtx(), apply.SelectBlockOffset()) + // Update Join conditions. + eq, left, right, other := newApply.LogicalJoin.ExtractOnCondition(newConds, outerChildGroup.Prop.Schema, innerChildGroup.Prop.Schema, false, false) + newApply.LogicalJoin.AppendJoinConds(eq, left, right, other) + + newApplyGroupExpr := memo.NewGroupExpr(newApply) + newApplyGroupExpr.SetChildren(outerChildGroup, old.Children[1].GetExpr().Children[0]) + return []*memo.GroupExpr{newApplyGroupExpr}, false, false, nil +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 9e719ca437685..3217a31568fa7 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1120,8 +1120,13 @@ func (lt *LogicalTopN) getPhysLimits() []PhysicalPlan { return ret } +<<<<<<< HEAD // Check if this prop's columns can match by items totally. func matchItems(p *property.PhysicalProperty, items []*ByItems) bool { +======= +// MatchItems checks if this prop's columns can match by items totally. +func MatchItems(p *property.PhysicalProperty, items []*util.ByItems) bool { +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) if len(items) < len(p.Items) { return false } diff --git a/planner/core/explain.go b/planner/core/explain.go index b4461bdef0a19..537ecab2f4c95 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/statistics" ) @@ -529,7 +530,212 @@ func formatWindowFuncDescs(buffer *bytes.Buffer, descs []*aggregation.WindowFunc if i != 0 { buffer.WriteString(", ") } +<<<<<<< HEAD buffer.WriteString(desc.String()) } return buffer } +======= + fmt.Fprintf(buffer, "%v->%v", desc, schema.Columns[winFuncStartIdx+i]) + } + return buffer +} + +// ExplainInfo implements Plan interface. +func (p *LogicalJoin) ExplainInfo() string { + buffer := bytes.NewBufferString(p.JoinType.String()) + if len(p.EqualConditions) > 0 { + fmt.Fprintf(buffer, ", equal:%v", p.EqualConditions) + } + if len(p.LeftConditions) > 0 { + fmt.Fprintf(buffer, ", left cond:%s", + expression.SortedExplainExpressionList(p.LeftConditions)) + } + if len(p.RightConditions) > 0 { + fmt.Fprintf(buffer, ", right cond:%s", + expression.SortedExplainExpressionList(p.RightConditions)) + } + if len(p.OtherConditions) > 0 { + fmt.Fprintf(buffer, ", other cond:%s", + expression.SortedExplainExpressionList(p.OtherConditions)) + } + return buffer.String() +} + +// ExplainInfo implements Plan interface. +func (p *LogicalAggregation) ExplainInfo() string { + buffer := bytes.NewBufferString("") + if len(p.GroupByItems) > 0 { + fmt.Fprintf(buffer, "group by:%s, ", + expression.SortedExplainExpressionList(p.GroupByItems)) + } + if len(p.AggFuncs) > 0 { + buffer.WriteString("funcs:") + for i, agg := range p.AggFuncs { + buffer.WriteString(aggregation.ExplainAggFunc(agg)) + if i+1 < len(p.AggFuncs) { + buffer.WriteString(", ") + } + } + } + return buffer.String() +} + +// ExplainInfo implements Plan interface. +func (p *LogicalProjection) ExplainInfo() string { + return expression.ExplainExpressionList(p.Exprs, p.schema) +} + +// ExplainInfo implements Plan interface. +func (p *LogicalSelection) ExplainInfo() string { + return string(expression.SortedExplainExpressionList(p.Conditions)) +} + +// ExplainInfo implements Plan interface. +func (p *LogicalApply) ExplainInfo() string { + return p.LogicalJoin.ExplainInfo() +} + +// ExplainInfo implements Plan interface. +func (p *LogicalTableDual) ExplainInfo() string { + return fmt.Sprintf("rowcount:%d", p.RowCount) +} + +// ExplainInfo implements Plan interface. +func (p *DataSource) ExplainInfo() string { + buffer := bytes.NewBufferString("") + tblName := p.tableInfo.Name.O + if p.TableAsName != nil && p.TableAsName.O != "" { + tblName = p.TableAsName.O + } + fmt.Fprintf(buffer, "table:%s", tblName) + if p.isPartition { + if pi := p.tableInfo.GetPartitionInfo(); pi != nil { + partitionName := pi.GetNameByID(p.physicalTableID) + fmt.Fprintf(buffer, ", partition:%s", partitionName) + } + } + return buffer.String() +} + +// ExplainInfo implements Plan interface. +func (p *LogicalUnionScan) ExplainInfo() string { + buffer := bytes.NewBufferString("") + fmt.Fprintf(buffer, "conds:%s", + expression.SortedExplainExpressionList(p.conditions)) + fmt.Fprintf(buffer, ", handle:%s", p.handleCol.ExplainInfo()) + return buffer.String() +} + +func explainByItems(buffer *bytes.Buffer, byItems []*util.ByItems) *bytes.Buffer { + for i, item := range byItems { + order := "asc" + if item.Desc { + order = "desc" + } + fmt.Fprintf(buffer, "%s:%s", item.Expr.ExplainInfo(), order) + if i+1 < len(byItems) { + buffer.WriteString(", ") + } + } + return buffer +} + +func explainNormalizedByItems(buffer *bytes.Buffer, byItems []*util.ByItems) *bytes.Buffer { + for i, item := range byItems { + order := "asc" + if item.Desc { + order = "desc" + } + fmt.Fprintf(buffer, "%s:%s", item.Expr.ExplainNormalizedInfo(), order) + if i+1 < len(byItems) { + buffer.WriteString(", ") + } + } + return buffer +} + +// ExplainInfo implements Plan interface. +func (p *LogicalSort) ExplainInfo() string { + buffer := bytes.NewBufferString("") + return explainByItems(buffer, p.ByItems).String() +} + +// ExplainInfo implements Plan interface. +func (p *LogicalTopN) ExplainInfo() string { + buffer := bytes.NewBufferString("") + buffer = explainByItems(buffer, p.ByItems) + fmt.Fprintf(buffer, ", offset:%v, count:%v", p.Offset, p.Count) + return buffer.String() +} + +// ExplainInfo implements Plan interface. +func (p *LogicalLimit) ExplainInfo() string { + return fmt.Sprintf("offset:%v, count:%v", p.Offset, p.Count) +} + +// ExplainInfo implements Plan interface. +func (p *LogicalTableScan) ExplainInfo() string { + buffer := bytes.NewBufferString(p.Source.ExplainInfo()) + if p.Source.handleCol != nil { + fmt.Fprintf(buffer, ", pk col:%s", p.Source.handleCol.ExplainInfo()) + } + if len(p.AccessConds) > 0 { + fmt.Fprintf(buffer, ", cond:%v", p.AccessConds) + } + return buffer.String() +} + +// ExplainInfo implements Plan interface. +func (p *LogicalIndexScan) ExplainInfo() string { + buffer := bytes.NewBufferString(p.Source.ExplainInfo()) + index := p.Index + if len(index.Columns) > 0 { + buffer.WriteString(", index:") + for i, idxCol := range index.Columns { + buffer.WriteString(idxCol.Name.O) + if i+1 < len(index.Columns) { + buffer.WriteString(", ") + } + } + } + if len(p.AccessConds) > 0 { + fmt.Fprintf(buffer, ", cond:%v", p.AccessConds) + } + return buffer.String() +} + +// ExplainInfo implements Plan interface. +func (p *TiKVSingleGather) ExplainInfo() string { + buffer := bytes.NewBufferString(p.Source.ExplainInfo()) + if p.IsIndexGather { + buffer.WriteString(", index:" + p.Index.Name.String()) + } + return buffer.String() +} + +// MetricTableTimeFormat is the time format for metric table explain and format. +const MetricTableTimeFormat = "2006-01-02 15:04:05.999" + +// ExplainInfo implements Plan interface. +func (p *PhysicalMemTable) ExplainInfo() string { + accessObject, operatorInfo := p.AccessObject(), p.OperatorInfo(false) + if len(operatorInfo) == 0 { + return accessObject + } + return accessObject + ", " + operatorInfo +} + +// AccessObject implements dataAccesser interface. +func (p *PhysicalMemTable) AccessObject() string { + return "table:" + p.Table.Name.O +} + +// OperatorInfo implements dataAccesser interface. +func (p *PhysicalMemTable) OperatorInfo(_ bool) string { + if p.Extractor != nil { + return p.Extractor.explainInfo(p) + } + return "" +} +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 0030510e1e31d..bb5808eddbb51 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -53,7 +53,11 @@ var invalidTask = &rootTask{cst: math.MaxFloat64} // getPropByOrderByItems will check if this sort property can be pushed or not. In order to simplify the problem, we only // consider the case that all expression are columns. +<<<<<<< HEAD func getPropByOrderByItems(items []*ByItems) (*property.PhysicalProperty, bool) { +======= +func GetPropByOrderByItems(items []*util.ByItems) (*property.PhysicalProperty, bool) { +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) propItems := make([]property.Item, 0, len(items)) for _, item := range items { col, ok := item.Expr.(*expression.Column) @@ -65,6 +69,32 @@ func getPropByOrderByItems(items []*ByItems) (*property.PhysicalProperty, bool) return &property.PhysicalProperty{Items: propItems}, true } +<<<<<<< HEAD +======= +// GetPropByOrderByItemsContainScalarFunc will check if this sort property can be pushed or not. In order to simplify the +// problem, we only consider the case that all expression are columns or some special scalar functions. +func GetPropByOrderByItemsContainScalarFunc(items []*util.ByItems) (*property.PhysicalProperty, bool, bool) { + propItems := make([]property.Item, 0, len(items)) + onlyColumn := true + for _, item := range items { + switch expr := item.Expr.(type) { + case *expression.Column: + propItems = append(propItems, property.Item{Col: expr, Desc: item.Desc}) + case *expression.ScalarFunction: + col, desc := expr.GetSingleColumn(item.Desc) + if col == nil { + return nil, false, false + } + propItems = append(propItems, property.Item{Col: col, Desc: desc}) + onlyColumn = false + default: + return nil, false, false + } + } + return &property.PhysicalProperty{Items: propItems}, true, onlyColumn +} + +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty) (task, error) { if !prop.IsEmpty() { return invalidTask, nil diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 002a849960f30..bc14f82529d21 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -20,6 +20,7 @@ import ( "math/bits" "reflect" "sort" + "strconv" "strings" "unicode" @@ -37,6 +38,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" @@ -73,6 +75,54 @@ func (la *LogicalAggregation) collectGroupByColumns() { } } +// aggOrderByResolver is currently resolving expressions of order by clause +// in aggregate function GROUP_CONCAT. +type aggOrderByResolver struct { + ctx sessionctx.Context + err error + args []ast.ExprNode + exprDepth int // exprDepth is the depth of current expression in expression tree. +} + +func (a *aggOrderByResolver) Enter(inNode ast.Node) (ast.Node, bool) { + a.exprDepth++ + switch n := inNode.(type) { + case *driver.ParamMarkerExpr: + if a.exprDepth == 1 { + _, isNull, isExpectedType := getUintFromNode(a.ctx, n) + // For constant uint expression in top level, it should be treated as position expression. + if !isNull && isExpectedType { + return expression.ConstructPositionExpr(n), true + } + } + } + return inNode, false +} + +func (a *aggOrderByResolver) Leave(inNode ast.Node) (ast.Node, bool) { + switch v := inNode.(type) { + case *ast.PositionExpr: + pos, isNull, err := expression.PosFromPositionExpr(a.ctx, v) + if err != nil { + a.err = err + } + if err != nil || isNull { + return inNode, false + } + if pos < 1 || pos > len(a.args) { + errPos := strconv.Itoa(pos) + if v.P != nil { + errPos = "?" + } + a.err = ErrUnknownColumn.FastGenByArgs(errPos, "order clause") + return inNode, false + } + ret := a.args[pos-1] + return ret, true + } + return inNode, true +} + func (b *PlanBuilder) buildAggregation(ctx context.Context, p LogicalPlan, aggFuncList []*ast.AggregateFuncExpr, gbyItems []expression.Expression) (LogicalPlan, map[int]int, error) { b.optFlag = b.optFlag | flagBuildKeyInfo b.optFlag = b.optFlag | flagPushDownAgg @@ -104,6 +154,27 @@ func (b *PlanBuilder) buildAggregation(ctx context.Context, p LogicalPlan, aggFu if err != nil { return nil, nil, err } + if aggFunc.Order != nil { + trueArgs := aggFunc.Args[:len(aggFunc.Args)-1] // the last argument is SEPARATOR, remote it. + resolver := &aggOrderByResolver{ + ctx: b.ctx, + args: trueArgs, + } + for _, byItem := range aggFunc.Order.Items { + resolver.exprDepth = 0 + resolver.err = nil + retExpr, _ := byItem.Expr.Accept(resolver) + if resolver.err != nil { + return nil, nil, errors.Trace(resolver.err) + } + newByItem, np, err := b.rewrite(ctx, retExpr.(ast.ExprNode), p, nil, true) + if err != nil { + return nil, nil, err + } + p = np + newFunc.OrderByItems = append(newFunc.OrderByItems, &util.ByItems{Expr: newByItem, Desc: byItem.Desc}) + } + } combined := false for j, oldFunc := range plan4Agg.AggFuncs { if oldFunc.Equal(b.ctx, newFunc) { @@ -959,6 +1030,7 @@ func (b *PlanBuilder) buildUnionAll(ctx context.Context, subPlan []LogicalPlan) return u } +<<<<<<< HEAD // ByItems wraps a "by" item. type ByItems struct { Expr expression.Expression @@ -978,6 +1050,8 @@ func (by *ByItems) Clone() *ByItems { return &ByItems{Expr: by.Expr.Clone(), Desc: by.Desc} } +======= +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) // itemTransformer transforms ParamMarkerExpr to PositionExpr in the context of ByItem type itemTransformer struct { } @@ -1001,8 +1075,13 @@ func (b *PlanBuilder) buildSort(ctx context.Context, p LogicalPlan, byItems []*a } else { b.curClause = orderByClause } +<<<<<<< HEAD sort := LogicalSort{}.Init(b.ctx) exprs := make([]*ByItems, 0, len(byItems)) +======= + sort := LogicalSort{}.Init(b.ctx, b.getSelectOffset()) + exprs := make([]*util.ByItems, 0, len(byItems)) +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) transformer := &itemTransformer{} for _, item := range byItems { newExpr, _ := item.Expr.Accept(transformer) @@ -1013,7 +1092,7 @@ func (b *PlanBuilder) buildSort(ctx context.Context, p LogicalPlan, byItems []*a } p = np - exprs = append(exprs, &ByItems{Expr: it, Desc: item.Desc}) + exprs = append(exprs, &util.ByItems{Expr: it, Desc: item.Desc}) } sort.ByItems = exprs sort.SetChildren(p) diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 9a2dc7e34fb5b..2e886f1cf071b 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/testleak" ) @@ -2583,7 +2584,7 @@ func (s *testPlanSuite) optimize(ctx context.Context, sql string) (PhysicalPlan, return p.(PhysicalPlan), stmt, err } -func byItemsToProperty(byItems []*ByItems) *property.PhysicalProperty { +func byItemsToProperty(byItems []*util.ByItems) *property.PhysicalProperty { pp := &property.PhysicalProperty{} for _, item := range byItems { pp.Items = append(pp.Items, property.Item{Col: item.Expr.(*expression.Column), Desc: item.Desc}) @@ -2669,7 +2670,7 @@ func (s *testPlanSuite) TestSkylinePruning(c *C) { _, err = lp.recursiveDeriveStats() c.Assert(err, IsNil) var ds *DataSource - var byItems []*ByItems + var byItems []*util.ByItems for ds == nil { switch v := lp.(type) { case *DataSource: @@ -2677,6 +2678,20 @@ func (s *testPlanSuite) TestSkylinePruning(c *C) { case *LogicalSort: byItems = v.ByItems lp = lp.Children()[0] +<<<<<<< HEAD +======= + case *LogicalProjection: + newItems := make([]*util.ByItems, 0, len(byItems)) + for _, col := range byItems { + idx := v.schema.ColumnIndex(col.Expr.(*expression.Column)) + switch expr := v.Exprs[idx].(type) { + case *expression.Column: + newItems = append(newItems, &util.ByItems{Expr: expr, Desc: col.Desc}) + } + } + byItems = newItems + lp = lp.Children()[0] +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) default: lp = lp.Children()[0] } diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 3cea589b8d8d3..b5f06dae49f06 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -647,7 +647,7 @@ type LogicalUnionAll struct { type LogicalSort struct { baseLogicalPlan - ByItems []*ByItems + ByItems []*util.ByItems } func (ls *LogicalSort) extractCorrelatedCols() []*expression.CorrelatedColumn { @@ -662,7 +662,7 @@ func (ls *LogicalSort) extractCorrelatedCols() []*expression.CorrelatedColumn { type LogicalTopN struct { baseLogicalPlan - ByItems []*ByItems + ByItems []*util.ByItems Offset uint64 Count uint64 } diff --git a/planner/core/pb_to_plan.go b/planner/core/pb_to_plan.go new file mode 100644 index 0000000000000..5ab1e76062b6f --- /dev/null +++ b/planner/core/pb_to_plan.go @@ -0,0 +1,277 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import ( + "strings" + + "github.com/pingcap/errors" + "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/planner/util" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tipb/go-tipb" +) + +// PBPlanBuilder uses to build physical plan from dag protocol buffers. +type PBPlanBuilder struct { + sctx sessionctx.Context + tps []*types.FieldType + is infoschema.InfoSchema +} + +// NewPBPlanBuilder creates a new pb plan builder. +func NewPBPlanBuilder(sctx sessionctx.Context, is infoschema.InfoSchema) *PBPlanBuilder { + return &PBPlanBuilder{sctx: sctx, is: is} +} + +// Build builds physical plan from dag protocol buffers. +func (b *PBPlanBuilder) Build(executors []*tipb.Executor) (p PhysicalPlan, err error) { + var src PhysicalPlan + for i := 0; i < len(executors); i++ { + curr, err := b.pbToPhysicalPlan(executors[i]) + if err != nil { + return nil, errors.Trace(err) + } + curr.SetChildren(src) + src = curr + } + _, src = b.predicatePushDown(src, nil) + return src, nil +} + +func (b *PBPlanBuilder) pbToPhysicalPlan(e *tipb.Executor) (p PhysicalPlan, err error) { + switch e.Tp { + case tipb.ExecType_TypeTableScan: + p, err = b.pbToTableScan(e) + case tipb.ExecType_TypeSelection: + p, err = b.pbToSelection(e) + case tipb.ExecType_TypeTopN: + p, err = b.pbToTopN(e) + case tipb.ExecType_TypeLimit: + p, err = b.pbToLimit(e) + case tipb.ExecType_TypeAggregation: + p, err = b.pbToAgg(e, false) + case tipb.ExecType_TypeStreamAgg: + p, err = b.pbToAgg(e, true) + default: + // TODO: Support other types. + err = errors.Errorf("this exec type %v doesn't support yet.", e.GetTp()) + } + return p, err +} + +func (b *PBPlanBuilder) pbToTableScan(e *tipb.Executor) (PhysicalPlan, error) { + tblScan := e.TblScan + tbl, ok := b.is.TableByID(tblScan.TableId) + if !ok { + return nil, infoschema.ErrTableNotExists.GenWithStack("Table which ID = %d does not exist.", tblScan.TableId) + } + dbInfo, ok := b.is.SchemaByTable(tbl.Meta()) + if !ok { + return nil, infoschema.ErrDatabaseNotExists.GenWithStack("Database of table ID = %d does not exist.", tblScan.TableId) + } + // Currently only support cluster table. + if !tbl.Type().IsClusterTable() { + return nil, errors.Errorf("table %s is not a cluster table", tbl.Meta().Name.L) + } + columns, err := b.convertColumnInfo(tbl.Meta(), tblScan.Columns) + if err != nil { + return nil, err + } + schema := b.buildTableScanSchema(tbl.Meta(), columns) + p := PhysicalMemTable{ + DBName: dbInfo.Name, + Table: tbl.Meta(), + Columns: columns, + }.Init(b.sctx, nil, 0) + p.SetSchema(schema) + if strings.ToUpper(p.Table.Name.O) == infoschema.ClusterTableSlowLog { + p.Extractor = &SlowQueryExtractor{} + } + return p, nil +} + +func (b *PBPlanBuilder) buildTableScanSchema(tblInfo *model.TableInfo, columns []*model.ColumnInfo) *expression.Schema { + schema := expression.NewSchema(make([]*expression.Column, 0, len(columns))...) + for _, col := range tblInfo.Columns { + for _, colInfo := range columns { + if col.ID != colInfo.ID { + continue + } + newCol := &expression.Column{ + UniqueID: b.sctx.GetSessionVars().AllocPlanColumnID(), + ID: col.ID, + RetType: &col.FieldType, + } + schema.Append(newCol) + } + } + return schema +} + +func (b *PBPlanBuilder) pbToSelection(e *tipb.Executor) (PhysicalPlan, error) { + conds, err := expression.PBToExprs(e.Selection.Conditions, b.tps, b.sctx.GetSessionVars().StmtCtx) + if err != nil { + return nil, err + } + p := PhysicalSelection{ + Conditions: conds, + }.Init(b.sctx, nil, 0) + return p, nil +} + +func (b *PBPlanBuilder) pbToTopN(e *tipb.Executor) (PhysicalPlan, error) { + topN := e.TopN + sc := b.sctx.GetSessionVars().StmtCtx + byItems := make([]*util.ByItems, 0, len(topN.OrderBy)) + for _, item := range topN.OrderBy { + expr, err := expression.PBToExpr(item.Expr, b.tps, sc) + if err != nil { + return nil, errors.Trace(err) + } + byItems = append(byItems, &util.ByItems{Expr: expr, Desc: item.Desc}) + } + p := PhysicalTopN{ + ByItems: byItems, + Count: topN.Limit, + }.Init(b.sctx, nil, 0) + return p, nil +} + +func (b *PBPlanBuilder) pbToLimit(e *tipb.Executor) (PhysicalPlan, error) { + p := PhysicalLimit{ + Count: e.Limit.Limit, + }.Init(b.sctx, nil, 0) + return p, nil +} + +func (b *PBPlanBuilder) pbToAgg(e *tipb.Executor, isStreamAgg bool) (PhysicalPlan, error) { + aggFuncs, groupBys, err := b.getAggInfo(e) + if err != nil { + return nil, errors.Trace(err) + } + schema := b.buildAggSchema(aggFuncs, groupBys) + baseAgg := basePhysicalAgg{ + AggFuncs: aggFuncs, + GroupByItems: groupBys, + } + baseAgg.schema = schema + var partialAgg PhysicalPlan + if isStreamAgg { + partialAgg = baseAgg.initForStream(b.sctx, nil, 0) + } else { + partialAgg = baseAgg.initForHash(b.sctx, nil, 0) + } + return partialAgg, nil +} + +func (b *PBPlanBuilder) buildAggSchema(aggFuncs []*aggregation.AggFuncDesc, groupBys []expression.Expression) *expression.Schema { + schema := expression.NewSchema(make([]*expression.Column, 0, len(aggFuncs)+len(groupBys))...) + for _, agg := range aggFuncs { + newCol := &expression.Column{ + UniqueID: b.sctx.GetSessionVars().AllocPlanColumnID(), + RetType: agg.RetTp, + } + schema.Append(newCol) + } + return schema +} + +func (b *PBPlanBuilder) getAggInfo(executor *tipb.Executor) ([]*aggregation.AggFuncDesc, []expression.Expression, error) { + var err error + aggFuncs := make([]*aggregation.AggFuncDesc, 0, len(executor.Aggregation.AggFunc)) + for _, expr := range executor.Aggregation.AggFunc { + aggFunc, err := aggregation.PBExprToAggFuncDesc(b.sctx, expr, b.tps) + if err != nil { + return nil, nil, errors.Trace(err) + } + aggFuncs = append(aggFuncs, aggFunc) + } + groupBys, err := expression.PBToExprs(executor.Aggregation.GetGroupBy(), b.tps, b.sctx.GetSessionVars().StmtCtx) + if err != nil { + return nil, nil, errors.Trace(err) + } + return aggFuncs, groupBys, nil +} + +func (b *PBPlanBuilder) convertColumnInfo(tblInfo *model.TableInfo, pbColumns []*tipb.ColumnInfo) ([]*model.ColumnInfo, error) { + columns := make([]*model.ColumnInfo, 0, len(pbColumns)) + tps := make([]*types.FieldType, 0, len(pbColumns)) + for _, col := range pbColumns { + found := false + for _, colInfo := range tblInfo.Columns { + if col.ColumnId == colInfo.ID { + columns = append(columns, colInfo) + tps = append(tps, colInfo.FieldType.Clone()) + found = true + break + } + } + if !found { + return nil, errors.Errorf("Column ID %v of table %v not found", col.ColumnId, tblInfo.Name.L) + } + } + b.tps = tps + return columns, nil +} + +func (b *PBPlanBuilder) predicatePushDown(p PhysicalPlan, predicates []expression.Expression) ([]expression.Expression, PhysicalPlan) { + if p == nil { + return predicates, p + } + switch p.(type) { + case *PhysicalMemTable: + memTable := p.(*PhysicalMemTable) + if memTable.Extractor == nil { + return predicates, p + } + names := make([]*types.FieldName, 0, len(memTable.Columns)) + for _, col := range memTable.Columns { + names = append(names, &types.FieldName{ + TblName: memTable.Table.Name, + ColName: col.Name, + OrigTblName: memTable.Table.Name, + OrigColName: col.Name, + }) + } + // Set the expression column unique ID. + // Since the expression is build from PB, It has not set the expression column ID yet. + schemaCols := memTable.schema.Columns + cols := expression.ExtractColumnsFromExpressions([]*expression.Column{}, predicates, nil) + for i := range cols { + cols[i].UniqueID = schemaCols[cols[i].Index].UniqueID + } + predicates = memTable.Extractor.Extract(b.sctx, memTable.schema, names, predicates) + return predicates, memTable + case *PhysicalSelection: + selection := p.(*PhysicalSelection) + conditions, child := b.predicatePushDown(p.Children()[0], selection.Conditions) + if len(conditions) > 0 { + selection.Conditions = conditions + selection.SetChildren(child) + return predicates, selection + } + return predicates, child + default: + if children := p.Children(); len(children) > 0 { + _, child := b.predicatePushDown(children[0], nil) + p.SetChildren(child) + } + return predicates, p + } +} diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index f0d5a44e437d7..db1643d669d51 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -659,5 +659,762 @@ func (s *testPlanSuite) TestIndexJoinHint(c *C) { c.Assert(warnings[0].Level, Equals, stmtctx.WarnLevelWarning) c.Assert(warnings[0].Err.Error(), Equals, output[i].Warning) } +<<<<<<< HEAD +======= + c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) + } +} + +func (s *testPlanSuite) TestAggregationHints(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + se, err := session.CreateSession4Test(store) + c.Assert(err, IsNil) + _, err = se.Execute(context.Background(), "use test") + c.Assert(err, IsNil) + + sessionVars := se.(sessionctx.Context).GetSessionVars() + sessionVars.HashAggFinalConcurrency = 1 + sessionVars.HashAggPartialConcurrency = 1 + + var input []struct { + SQL string + AggPushDown bool + } + var output []struct { + SQL string + Best string + Warning string + } + s.testData.GetTestCases(c, &input, &output) + ctx := context.Background() + for i, test := range input { + comment := Commentf("case:%v sql:%s", i, test) + se.GetSessionVars().StmtCtx.SetWarnings(nil) + se.GetSessionVars().AllowAggPushDown = test.AggPushDown + + stmt, err := s.ParseOneStmt(test.SQL, "", "") + c.Assert(err, IsNil, comment) + + p, _, err := planner.Optimize(ctx, se, stmt, s.is) + c.Assert(err, IsNil) + warnings := se.GetSessionVars().StmtCtx.GetWarnings() + + s.testData.OnRecord(func() { + output[i].SQL = test.SQL + output[i].Best = core.ToString(p) + if len(warnings) > 0 { + output[i].Warning = warnings[0].Err.Error() + } + }) + c.Assert(core.ToString(p), Equals, output[i].Best, comment) + if output[i].Warning == "" { + c.Assert(len(warnings), Equals, 0, comment) + } else { + c.Assert(len(warnings), Equals, 1, comment) + c.Assert(warnings[0].Level, Equals, stmtctx.WarnLevelWarning, comment) + c.Assert(warnings[0].Err.Error(), Equals, output[i].Warning, comment) + } + } +} + +func (s *testPlanSuite) TestAggToCopHint(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists ta") + tk.MustExec("create table ta(a int, b int, index(a))") + + var ( + input []string + output []struct { + SQL string + Best string + Warning string + } + ) + s.testData.GetTestCases(c, &input, &output) + + ctx := context.Background() + is := domain.GetDomain(tk.Se).InfoSchema() + for i, test := range input { + comment := Commentf("case:%v sql:%s", i, test) + s.testData.OnRecord(func() { + output[i].SQL = test + }) + c.Assert(test, Equals, output[i].SQL, comment) + + tk.Se.GetSessionVars().StmtCtx.SetWarnings(nil) + + stmt, err := s.ParseOneStmt(test, "", "") + c.Assert(err, IsNil, comment) + + p, _, err := planner.Optimize(ctx, tk.Se, stmt, is) + c.Assert(err, IsNil) + planString := core.ToString(p) + s.testData.OnRecord(func() { + output[i].Best = planString + }) + c.Assert(planString, Equals, output[i].Best, comment) + + warnings := tk.Se.GetSessionVars().StmtCtx.GetWarnings() + s.testData.OnRecord(func() { + if len(warnings) > 0 { + output[i].Warning = warnings[0].Err.Error() + } + }) + if output[i].Warning == "" { + c.Assert(len(warnings), Equals, 0, comment) + } else { + c.Assert(len(warnings), Equals, 1, comment) + c.Assert(warnings[0].Level, Equals, stmtctx.WarnLevelWarning, comment) + c.Assert(warnings[0].Err.Error(), Equals, output[i].Warning, comment) + } + } +} + +func (s *testPlanSuite) TestPushdownDistinctEnable(c *C) { + defer testleak.AfterTest(c)() + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + s.testData.GetTestCases(c, &input, &output) + vars := []string{ + fmt.Sprintf("set @@session.%s = 1", variable.TiDBOptDistinctAggPushDown), + "set session tidb_opt_agg_push_down = 1", + } + s.doTestPushdownDistinct(c, vars, input, output) +} + +func (s *testPlanSuite) TestPushdownDistinctDisable(c *C) { + defer testleak.AfterTest(c)() + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + s.testData.GetTestCases(c, &input, &output) + vars := []string{ + fmt.Sprintf("set @@session.%s = 0", variable.TiDBOptDistinctAggPushDown), + "set session tidb_opt_agg_push_down = 1", + } + s.doTestPushdownDistinct(c, vars, input, output) +} + +func (s *testPlanSuite) TestPushdownDistinctEnableAggPushDownDisable(c *C) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + s.testData.GetTestCases(c, &input, &output) + vars := []string{ + fmt.Sprintf("set @@session.%s = 1", variable.TiDBOptDistinctAggPushDown), + "set session tidb_opt_agg_push_down = 0", + } + s.doTestPushdownDistinct(c, vars, input, output) +} + +func (s *testPlanSuite) doTestPushdownDistinct(c *C, vars, input []string, output []struct { + SQL string + Plan []string + Result []string +}) { + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test") + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, c int, index(c))") + tk.MustExec("insert into t values (1, 1, 1), (1, 1, 3), (1, 2, 3), (2, 1, 3), (1, 2, NULL);") + + tk.MustExec("drop table if exists pt") + tk.MustExec(`CREATE TABLE pt (a int, b int) PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (2), + PARTITION p1 VALUES LESS THAN (100) + );`) + + tk.MustExec("drop table if exists ta") + tk.MustExec("create table ta(a int);") + tk.MustExec("insert into ta values(1), (1);") + tk.MustExec("drop table if exists tb") + tk.MustExec("create table tb(a int);") + tk.MustExec("insert into tb values(1), (1);") + + tk.MustExec("set session sql_mode=''") + tk.MustExec(fmt.Sprintf("set session %s=1", variable.TiDBHashAggPartialConcurrency)) + tk.MustExec(fmt.Sprintf("set session %s=1", variable.TiDBHashAggFinalConcurrency)) + + for _, v := range vars { + tk.MustExec(v) + } + + for i, ts := range input { + s.testData.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + ts).Rows()) + output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + }) + tk.MustQuery("explain " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) + } +} + +func (s *testPlanSuite) TestGroupConcatOrderby(c *C) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + s.testData.GetTestCases(c, &input, &output) + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists test;") + tk.MustExec("create table test(id int, name int)") + tk.MustExec("insert into test values(1, 10);") + tk.MustExec("insert into test values(1, 20);") + tk.MustExec("insert into test values(1, 30);") + tk.MustExec("insert into test values(2, 20);") + tk.MustExec("insert into test values(3, 200);") + tk.MustExec("insert into test values(3, 500);") + + tk.MustExec("drop table if exists ptest;") + tk.MustExec("CREATE TABLE ptest (id int,name int) PARTITION BY RANGE ( id ) " + + "(PARTITION `p0` VALUES LESS THAN (2), PARTITION `p1` VALUES LESS THAN (11))") + tk.MustExec("insert into ptest select * from test;") + tk.MustExec(fmt.Sprintf("set session tidb_opt_distinct_agg_push_down = %v", 1)) + tk.MustExec(fmt.Sprintf("set session tidb_opt_agg_push_down = %v", 1)) + + for i, ts := range input { + s.testData.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + ts).Rows()) + output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + }) + tk.MustQuery("explain " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Check(testkit.Rows(output[i].Result...)) + } +} + +func (s *testPlanSuite) TestHintAlias(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + se, err := session.CreateSession4Test(store) + c.Assert(err, IsNil) + _, err = se.Execute(context.Background(), "use test") + c.Assert(err, IsNil) + + tests := []struct { + sql1 string + sql2 string + }{ + { + sql1: "select /*+ TIDB_SMJ(t1) */ t1.a, t1.b from t t1, (select /*+ TIDB_INLJ(t3) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", + sql2: "select /*+ MERGE_JOIN(t1) */ t1.a, t1.b from t t1, (select /*+ INL_JOIN(t3) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", + }, + { + sql1: "select /*+ TIDB_HJ(t1) */ t1.a, t1.b from t t1, (select /*+ TIDB_SMJ(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", + sql2: "select /*+ HASH_JOIN(t1) */ t1.a, t1.b from t t1, (select /*+ MERGE_JOIN(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", + }, + { + sql1: "select /*+ TIDB_INLJ(t1) */ t1.a, t1.b from t t1, (select /*+ TIDB_HJ(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", + sql2: "select /*+ INL_JOIN(t1) */ t1.a, t1.b from t t1, (select /*+ HASH_JOIN(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", + }, + } + ctx := context.TODO() + for i, tt := range tests { + comment := Commentf("case:%v sql1:%s sql2:%s", i, tt.sql1, tt.sql2) + stmt1, err := s.ParseOneStmt(tt.sql1, "", "") + c.Assert(err, IsNil, comment) + stmt2, err := s.ParseOneStmt(tt.sql2, "", "") + c.Assert(err, IsNil, comment) + + p1, _, err := planner.Optimize(ctx, se, stmt1, s.is) + c.Assert(err, IsNil) + p2, _, err := planner.Optimize(ctx, se, stmt2, s.is) + c.Assert(err, IsNil) + + c.Assert(core.ToString(p1), Equals, core.ToString(p2)) + } +} + +func (s *testPlanSuite) TestIndexHint(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + se, err := session.CreateSession4Test(store) + c.Assert(err, IsNil) + _, err = se.Execute(context.Background(), "use test") + c.Assert(err, IsNil) + + var input []string + var output []struct { + SQL string + Best string + HasWarn bool + Hints string + } + s.testData.GetTestCases(c, &input, &output) + ctx := context.Background() + for i, test := range input { + comment := Commentf("case:%v sql:%s", i, test) + se.GetSessionVars().StmtCtx.SetWarnings(nil) + + stmt, err := s.ParseOneStmt(test, "", "") + c.Assert(err, IsNil, comment) + + p, _, err := planner.Optimize(ctx, se, stmt, s.is) + c.Assert(err, IsNil) + s.testData.OnRecord(func() { + output[i].SQL = test + output[i].Best = core.ToString(p) + output[i].HasWarn = len(se.GetSessionVars().StmtCtx.GetWarnings()) > 0 + output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) + }) + c.Assert(core.ToString(p), Equals, output[i].Best, comment) + warnings := se.GetSessionVars().StmtCtx.GetWarnings() + if output[i].HasWarn { + c.Assert(warnings, HasLen, 1, comment) + } else { + c.Assert(warnings, HasLen, 0, comment) + } + c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) + } +} + +func (s *testPlanSuite) TestIndexMergeHint(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + se, err := session.CreateSession4Test(store) + c.Assert(err, IsNil) + _, err = se.Execute(context.Background(), "use test") + c.Assert(err, IsNil) + + var input []string + var output []struct { + SQL string + Best string + HasWarn bool + Hints string + } + s.testData.GetTestCases(c, &input, &output) + ctx := context.Background() + for i, test := range input { + comment := Commentf("case:%v sql:%s", i, test) + se.GetSessionVars().StmtCtx.SetWarnings(nil) + stmt, err := s.ParseOneStmt(test, "", "") + c.Assert(err, IsNil, comment) + sctx := se.(sessionctx.Context) + err = executor.ResetContextOfStmt(sctx, stmt) + c.Assert(err, IsNil) + p, _, err := planner.Optimize(ctx, se, stmt, s.is) + c.Assert(err, IsNil) + s.testData.OnRecord(func() { + output[i].SQL = test + output[i].Best = core.ToString(p) + output[i].HasWarn = len(se.GetSessionVars().StmtCtx.GetWarnings()) > 0 + output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) + }) + c.Assert(core.ToString(p), Equals, output[i].Best, comment) + warnings := se.GetSessionVars().StmtCtx.GetWarnings() + if output[i].HasWarn { + c.Assert(warnings, HasLen, 1, comment) + } else { + c.Assert(warnings, HasLen, 0, comment) + } + c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) + } +} + +func (s *testPlanSuite) TestQueryBlockHint(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + se, err := session.CreateSession4Test(store) + c.Assert(err, IsNil) + _, err = se.Execute(context.Background(), "use test") + c.Assert(err, IsNil) + + var input []string + var output []struct { + SQL string + Plan string + Hints string + } + s.testData.GetTestCases(c, &input, &output) + ctx := context.TODO() + for i, tt := range input { + comment := Commentf("case:%v sql: %s", i, tt) + stmt, err := s.ParseOneStmt(tt, "", "") + c.Assert(err, IsNil, comment) + + p, _, err := planner.Optimize(ctx, se, stmt, s.is) + c.Assert(err, IsNil, comment) + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = core.ToString(p) + output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) + }) + c.Assert(core.ToString(p), Equals, output[i].Plan, comment) + c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) + } +} + +func (s *testPlanSuite) TestInlineProjection(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + se, err := session.CreateSession4Test(store) + c.Assert(err, IsNil) + ctx := context.Background() + _, err = se.Execute(ctx, "use test") + c.Assert(err, IsNil) + _, err = se.Execute(ctx, `drop table if exists test.t1, test.t2;`) + c.Assert(err, IsNil) + _, err = se.Execute(ctx, `create table test.t1(a bigint, b bigint, index idx_a(a), index idx_b(b));`) + c.Assert(err, IsNil) + _, err = se.Execute(ctx, `create table test.t2(a bigint, b bigint, index idx_a(a), index idx_b(b));`) + c.Assert(err, IsNil) + + var input []string + var output []struct { + SQL string + Plan string + Hints string + } + is := domain.GetDomain(se).InfoSchema() + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + comment := Commentf("case:%v sql: %s", i, tt) + stmt, err := s.ParseOneStmt(tt, "", "") + c.Assert(err, IsNil, comment) + + p, _, err := planner.Optimize(ctx, se, stmt, is) + c.Assert(err, IsNil, comment) + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = core.ToString(p) + output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) + }) + c.Assert(core.ToString(p), Equals, output[i].Plan, comment) + c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) + } +} + +func (s *testPlanSuite) TestDAGPlanBuilderSplitAvg(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + se, err := session.CreateSession4Test(store) + c.Assert(err, IsNil) + _, err = se.Execute(context.Background(), "use test") + c.Assert(err, IsNil) + tests := []struct { + sql string + plan string + }{ + { + sql: "select avg(a),avg(b),avg(c) from t", + plan: "TableReader(Table(t)->StreamAgg)->StreamAgg", + }, + { + sql: "select /*+ HASH_AGG() */ avg(a),avg(b),avg(c) from t", + plan: "TableReader(Table(t)->HashAgg)->HashAgg", + }, + } + + for _, tt := range tests { + comment := Commentf("for %s", tt.sql) + stmt, err := s.ParseOneStmt(tt.sql, "", "") + c.Assert(err, IsNil, comment) + + core.Preprocess(se, stmt, s.is) + p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) + c.Assert(err, IsNil, comment) + + c.Assert(core.ToString(p), Equals, tt.plan, comment) + root, ok := p.(core.PhysicalPlan) + if !ok { + continue + } + testDAGPlanBuilderSplitAvg(c, root) + } +} + +func testDAGPlanBuilderSplitAvg(c *C, root core.PhysicalPlan) { + if p, ok := root.(*core.PhysicalTableReader); ok { + if p.TablePlans != nil { + baseAgg := p.TablePlans[len(p.TablePlans)-1] + if agg, ok := baseAgg.(*core.PhysicalHashAgg); ok { + for i, aggfunc := range agg.AggFuncs { + c.Assert(agg.Schema().Columns[i].RetType, Equals, aggfunc.RetTp) + } + } + if agg, ok := baseAgg.(*core.PhysicalStreamAgg); ok { + for i, aggfunc := range agg.AggFuncs { + c.Assert(agg.Schema().Columns[i].RetType, Equals, aggfunc.RetTp) + } + } + } + } + + childs := root.Children() + if childs == nil { + return + } + for _, son := range childs { + testDAGPlanBuilderSplitAvg(c, son) + } +} + +func (s *testPlanSuite) TestIndexJoinHint(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + se, err := session.CreateSession4Test(store) + c.Assert(err, IsNil) + ctx := context.Background() + _, err = se.Execute(ctx, "use test") + c.Assert(err, IsNil) + _, err = se.Execute(ctx, `drop table if exists test.t1, test.t2, test.t;`) + c.Assert(err, IsNil) + _, err = se.Execute(ctx, `create table test.t1(a bigint, b bigint, index idx_a(a), index idx_b(b));`) + c.Assert(err, IsNil) + _, err = se.Execute(ctx, `create table test.t2(a bigint, b bigint, index idx_a(a), index idx_b(b));`) + c.Assert(err, IsNil) + _, err = se.Execute(ctx, "CREATE TABLE `t` ( `a` bigint(20) NOT NULL, `b` tinyint(1) DEFAULT NULL, `c` datetime DEFAULT NULL, `d` int(10) unsigned DEFAULT NULL, `e` varchar(20) DEFAULT NULL, `f` double DEFAULT NULL, `g` decimal(30,5) DEFAULT NULL, `h` float DEFAULT NULL, `i` date DEFAULT NULL, `j` timestamp NULL DEFAULT NULL, PRIMARY KEY (`a`), UNIQUE KEY `b` (`b`), KEY `c` (`c`,`d`,`e`), KEY `f` (`f`), KEY `g` (`g`,`h`), KEY `g_2` (`g`), UNIQUE KEY `g_3` (`g`), KEY `i` (`i`) );") + c.Assert(err, IsNil) + var input []string + var output []struct { + SQL string + Plan string + } + is := domain.GetDomain(se).InfoSchema() + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + comment := Commentf("case:%v sql: %s", i, tt) + stmt, err := s.ParseOneStmt(tt, "", "") + c.Assert(err, IsNil, comment) + p, _, err := planner.Optimize(ctx, se, stmt, is) + c.Assert(err, IsNil, comment) + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = core.ToString(p) + }) + c.Assert(core.ToString(p), Equals, output[i].Plan, comment) + } +} + +func (s *testPlanSuite) TestDAGPlanBuilderWindow(c *C) { + defer testleak.AfterTest(c)() + var input []string + var output []struct { + SQL string + Best string + } + s.testData.GetTestCases(c, &input, &output) + vars := []string{ + "set @@session.tidb_window_concurrency = 1", + } + s.doTestDAGPlanBuilderWindow(c, vars, input, output) +} + +func (s *testPlanSuite) TestDAGPlanBuilderWindowParallel(c *C) { + defer testleak.AfterTest(c)() + var input []string + var output []struct { + SQL string + Best string + } + s.testData.GetTestCases(c, &input, &output) + vars := []string{ + "set @@session.tidb_window_concurrency = 4", + } + s.doTestDAGPlanBuilderWindow(c, vars, input, output) +} + +func (s *testPlanSuite) doTestDAGPlanBuilderWindow(c *C, vars, input []string, output []struct { + SQL string + Best string +}) { + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + se, err := session.CreateSession4Test(store) + c.Assert(err, IsNil) + ctx := context.Background() + _, err = se.Execute(ctx, "use test") + c.Assert(err, IsNil) + + for _, v := range vars { + _, err = se.Execute(ctx, v) + c.Assert(err, IsNil) + } + + for i, tt := range input { + comment := Commentf("case:%v sql:%s", i, tt) + stmt, err := s.ParseOneStmt(tt, "", "") + c.Assert(err, IsNil, comment) + + err = se.NewTxn(context.Background()) + c.Assert(err, IsNil) + p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) + c.Assert(err, IsNil) + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Best = core.ToString(p) + }) + c.Assert(core.ToString(p), Equals, output[i].Best, comment) + } +} + +func (s *testPlanSuite) TestNominalSort(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + defer func() { + dom.Close() + store.Close() + }() + tk.MustExec("use test") + var input []string + var output []struct { + SQL string + Plan []string + Result []string + } + tk.MustExec("create table t (a int, b int, index idx_a(a), index idx_b(b))") + tk.MustExec("insert into t values(1, 1)") + tk.MustExec("insert into t values(1, 2)") + tk.MustExec("insert into t values(2, 4)") + tk.MustExec("insert into t values(3, 5)") + s.testData.GetTestCases(c, &input, &output) + for i, ts := range input { + s.testData.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + ts).Rows()) + output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Rows()) + }) + tk.MustQuery("explain " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Check(testkit.Rows(output[i].Result...)) + } +} + +func (s *testPlanSuite) TestHintFromDiffDatabase(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + se, err := session.CreateSession4Test(store) + c.Assert(err, IsNil) + ctx := context.Background() + _, err = se.Execute(ctx, "use test") + c.Assert(err, IsNil) + _, err = se.Execute(ctx, `drop table if exists test.t1`) + c.Assert(err, IsNil) + _, err = se.Execute(ctx, `create table test.t1(a bigint, index idx_a(a));`) + c.Assert(err, IsNil) + _, err = se.Execute(ctx, `create table test.t2(a bigint, index idx_a(a));`) + c.Assert(err, IsNil) + + _, err = se.Execute(ctx, "drop database if exists test2") + c.Assert(err, IsNil) + _, err = se.Execute(ctx, "create database test2") + c.Assert(err, IsNil) + _, err = se.Execute(ctx, "use test2") + c.Assert(err, IsNil) + + var input []string + var output []struct { + SQL string + Plan string + } + is := domain.GetDomain(se).InfoSchema() + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + comment := Commentf("case:%v sql: %s", i, tt) + stmt, err := s.ParseOneStmt(tt, "", "") + c.Assert(err, IsNil, comment) + p, _, err := planner.Optimize(ctx, se, stmt, is) + c.Assert(err, IsNil, comment) + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = core.ToString(p) + }) + c.Assert(core.ToString(p), Equals, output[i].Plan, comment) +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index ffabc868fe4dd..8d54f8d1f7d36 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -19,6 +19,11 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/planner/property" +<<<<<<< HEAD +======= + "github.com/pingcap/tidb/planner/util" + "github.com/pingcap/tidb/sessionctx" +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" @@ -195,7 +200,7 @@ type PhysicalProjection struct { type PhysicalTopN struct { basePhysicalPlan - ByItems []*ByItems + ByItems []*util.ByItems Offset uint64 Count uint64 } @@ -353,13 +358,22 @@ type PhysicalStreamAgg struct { type PhysicalSort struct { basePhysicalPlan - ByItems []*ByItems + ByItems []*util.ByItems } // NominalSort asks sort properties for its child. It is a fake operator that will not // appear in final physical operator tree. type NominalSort struct { basePhysicalPlan +<<<<<<< HEAD +======= + + // These two fields are used to switch ScalarFunctions to Constants. For these + // NominalSorts, we need to converted to Projections check if the ScalarFunctions + // are out of bounds. (issue #11653) + ByItems []*util.ByItems + OnlyColumn bool +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } // PhysicalUnionScan represents a union scan operator. diff --git a/planner/core/plan.go b/planner/core/plan.go index 4540a7480af7b..79705ae6d251e 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" @@ -56,9 +57,13 @@ func enforceProperty(p *property.PhysicalProperty, tsk task, ctx sessionctx.Cont } tsk = finishCopTask(ctx, tsk) sortReqProp := &property.PhysicalProperty{TaskTp: property.RootTaskType, Items: p.Items, ExpectedCnt: math.MaxFloat64} +<<<<<<< HEAD sort := PhysicalSort{ByItems: make([]*ByItems, 0, len(p.Items))}.Init(ctx, tsk.plan().statsInfo(), sortReqProp) +======= + sort := PhysicalSort{ByItems: make([]*util.ByItems, 0, len(p.Items))}.Init(ctx, tsk.plan().statsInfo(), tsk.plan().SelectBlockOffset(), sortReqProp) +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) for _, col := range p.Items { - sort.ByItems = append(sort.ByItems, &ByItems{col.Col, col.Desc}) + sort.ByItems = append(sort.ByItems, &util.ByItems{Expr: col.Col, Desc: col.Desc}) } return sort.attach2Task(tsk) } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 52d5530e08d63..9faf2912c380d 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1377,12 +1377,23 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, } return substitutePlaceHolderDual(physical, p), nil } +<<<<<<< HEAD return p, nil } func substitutePlaceHolderDual(src PhysicalPlan, dst PhysicalPlan) PhysicalPlan { if dual, ok := src.(*PhysicalTableDual); ok && dual.placeHolder { return dst +======= + if show.Tp == ast.ShowVariables || show.Tp == ast.ShowStatus { + b.curClause = orderByClause + orderByCol := np.Schema().Columns[0].Clone().(*expression.Column) + sort := LogicalSort{ + ByItems: []*util.ByItems{{Expr: orderByCol}}, + }.Init(b.ctx, b.getSelectOffset()) + sort.SetChildren(np) + np = sort +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } for i, child := range src.Children() { newChild := substitutePlaceHolderDual(child, dst) diff --git a/planner/core/property_cols_prune.go b/planner/core/property_cols_prune.go index 1ec2933485ec8..4c22ac39fab5f 100644 --- a/planner/core/property_cols_prune.go +++ b/planner/core/property_cols_prune.go @@ -15,6 +15,7 @@ package core import ( "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/planner/util" ) func (ds *DataSource) preparePossibleProperties() [][]*expression.Column { @@ -57,7 +58,7 @@ func (p *LogicalTopN) preparePossibleProperties() [][]*expression.Column { return [][]*expression.Column{propCols} } -func getPossiblePropertyFromByItems(items []*ByItems) []*expression.Column { +func getPossiblePropertyFromByItems(items []*util.ByItems) []*expression.Column { cols := make([]*expression.Column, 0, len(items)) for _, item := range items { if col, ok := item.Expr.(*expression.Column); ok { diff --git a/planner/core/resolve_indices.go b/planner/core/resolve_indices.go index 2340e747bb731..1679f35a982ea 100644 --- a/planner/core/resolve_indices.go +++ b/planner/core/resolve_indices.go @@ -290,6 +290,12 @@ func (p *basePhysicalAgg) ResolveIndices() (err error) { return err } } + for _, byItem := range aggFun.OrderByItems { + byItem.Expr, err = byItem.Expr.ResolveIndices(p.children[0].Schema()) + if err != nil { + return err + } + } } for i, item := range p.GroupByItems { p.GroupByItems[i], err = item.ResolveIndices(p.children[0].Schema()) diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 6f7488667b0ac..22d40545a1165 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -35,7 +35,10 @@ type aggregationPushDownSolver struct { // It's easy to see that max, min, first row is decomposable, no matter whether it's distinct, but sum(distinct) and // count(distinct) is not. // Currently we don't support avg and concat. -func (a *aggregationPushDownSolver) isDecomposable(fun *aggregation.AggFuncDesc) bool { +func (a *aggregationPushDownSolver) isDecomposableWithJoin(fun *aggregation.AggFuncDesc) bool { + if len(fun.OrderByItems) > 0 { + return false + } switch fun.Name { case ast.AggFuncAvg, ast.AggFuncGroupConcat: // TODO: Support avg push down. @@ -49,6 +52,22 @@ func (a *aggregationPushDownSolver) isDecomposable(fun *aggregation.AggFuncDesc) } } +func (a *aggregationPushDownSolver) isDecomposableWithUnion(fun *aggregation.AggFuncDesc) bool { + if len(fun.OrderByItems) > 0 { + return false + } + switch fun.Name { + case ast.AggFuncGroupConcat, ast.AggFuncVarPop, ast.AggFuncJsonObjectAgg: + return false + case ast.AggFuncMax, ast.AggFuncMin, ast.AggFuncFirstRow: + return true + case ast.AggFuncSum, ast.AggFuncCount, ast.AggFuncAvg: + return true + default: + return false + } +} + // getAggFuncChildIdx gets which children it belongs to, 0 stands for left, 1 stands for right, -1 stands for both. func (a *aggregationPushDownSolver) getAggFuncChildIdx(aggFunc *aggregation.AggFuncDesc, schema *expression.Schema) int { fromLeft, fromRight := false, false @@ -76,7 +95,7 @@ func (a *aggregationPushDownSolver) collectAggFuncs(agg *LogicalAggregation, joi valid = true leftChild := join.children[0] for _, aggFunc := range agg.AggFuncs { - if !a.isDecomposable(aggFunc) { + if !a.isDecomposableWithJoin(aggFunc) { return false, nil, nil } index := a.getAggFuncChildIdx(aggFunc, leftChild.Schema()) @@ -378,12 +397,21 @@ func (a *aggregationPushDownSolver) aggPushDown(p LogicalPlan) (_ LogicalPlan, e projChild := proj.children[0] agg.SetChildren(projChild) } else if union, ok1 := child.(*LogicalUnionAll); ok1 { +<<<<<<< HEAD var gbyCols []*expression.Column gbyCols = expression.ExtractColumnsFromExpressions(gbyCols, agg.GroupByItems, nil) pushedAgg, err := a.makeNewAgg(agg.ctx, agg.AggFuncs, gbyCols) if err != nil { return nil, err } +======= + for _, aggFunc := range agg.AggFuncs { + if !a.isDecomposableWithUnion(aggFunc) { + return p, nil + } + } + pushedAgg := a.splitPartialAgg(agg) +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) newChildren := make([]LogicalPlan, 0, len(union.children)) for _, child := range union.children { newChild := a.pushAggCrossUnion(pushedAgg, union.Schema(), child) diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 595cd6d04e30e..d5aa70d06d225 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -23,7 +23,11 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" +<<<<<<< HEAD "github.com/pingcap/tidb/infoschema" +======= + "github.com/pingcap/tidb/planner/util" +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) "github.com/pingcap/tidb/types" ) @@ -120,6 +124,10 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column) var selfUsedCols []*expression.Column for _, aggrFunc := range la.AggFuncs { selfUsedCols = expression.ExtractColumnsFromExpressions(selfUsedCols, aggrFunc.Args, nil) + + var cols []*expression.Column + aggrFunc.OrderByItems, cols = pruneByItems(aggrFunc.OrderByItems) + selfUsedCols = append(selfUsedCols, cols...) } if len(la.AggFuncs) == 0 { // If all the aggregate functions are pruned, we should add an aggregate function to keep the correctness. @@ -154,31 +162,61 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column) return child.PruneColumns(selfUsedCols) } +<<<<<<< HEAD // PruneColumns implements LogicalPlan interface. func (ls *LogicalSort) PruneColumns(parentUsedCols []*expression.Column) error { child := ls.children[0] for i := len(ls.ByItems) - 1; i >= 0; i-- { cols := expression.ExtractColumns(ls.ByItems[i].Expr) +======= +func pruneByItems(old []*util.ByItems) (new []*util.ByItems, parentUsedCols []*expression.Column) { + new = make([]*util.ByItems, 0, len(old)) + for _, byItem := range old { + cols := expression.ExtractColumns(byItem.Expr) +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) if len(cols) == 0 { - if !expression.IsRuntimeConstExpr(ls.ByItems[i].Expr) { - continue + if !expression.IsRuntimeConstExpr(byItem.Expr) { + new = append(new, byItem) } - ls.ByItems = append(ls.ByItems[:i], ls.ByItems[i+1:]...) - } else if ls.ByItems[i].Expr.GetType().Tp == mysql.TypeNull { - ls.ByItems = append(ls.ByItems[:i], ls.ByItems[i+1:]...) + } else if byItem.Expr.GetType().Tp == mysql.TypeNull { + // do nothing, should be filtered } else { parentUsedCols = append(parentUsedCols, cols...) + new = append(new, byItem) } } + return +} + +// PruneColumns implements LogicalPlan interface. +// If any expression can view as a constant in execution stage, such as correlated column, constant, +// we do prune them. Note that we can't prune the expressions contain non-deterministic functions, such as rand(). +func (ls *LogicalSort) PruneColumns(parentUsedCols []*expression.Column) error { + child := ls.children[0] + var cols []*expression.Column + ls.ByItems, cols = pruneByItems(ls.ByItems) + parentUsedCols = append(parentUsedCols, cols...) return child.PruneColumns(parentUsedCols) } // PruneColumns implements LogicalPlan interface. +<<<<<<< HEAD func (p *LogicalUnionAll) PruneColumns(parentUsedCols []*expression.Column) error { used, err := getUsedList(parentUsedCols, p.schema) if err != nil { return err } +======= +// If any expression can view as a constant in execution stage, such as correlated column, constant, +// we do prune them. Note that we can't prune the expressions contain non-deterministic functions, such as rand(). +func (lt *LogicalTopN) PruneColumns(parentUsedCols []*expression.Column) error { + child := lt.children[0] + var cols []*expression.Column + lt.ByItems, cols = pruneByItems(lt.ByItems) + parentUsedCols = append(parentUsedCols, cols...) + return child.PruneColumns(parentUsedCols) +} +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) hasBeenUsed := false for i := range used { diff --git a/planner/core/rule_inject_extra_projection.go b/planner/core/rule_inject_extra_projection.go index a2bcbcf28e3eb..7101ef59b38cc 100644 --- a/planner/core/rule_inject_extra_projection.go +++ b/planner/core/rule_inject_extra_projection.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" ) @@ -79,6 +80,10 @@ func injectProjBelowAgg(aggPlan PhysicalPlan, aggFuncs []*aggregation.AggFuncDes _, isScalarFunc := arg.(*expression.ScalarFunction) hasScalarFunc = hasScalarFunc || isScalarFunc } + for _, byItem := range aggFuncs[i].OrderByItems { + _, isScalarFunc := byItem.Expr.(*expression.ScalarFunction) + hasScalarFunc = hasScalarFunc || isScalarFunc + } } for i := 0; !hasScalarFunc && i < len(groupByItems); i++ { _, isScalarFunc := groupByItems[i].(*expression.ScalarFunction) @@ -108,6 +113,20 @@ func injectProjBelowAgg(aggPlan PhysicalPlan, aggFuncs []*aggregation.AggFuncDes f.Args[i] = newArg cursor++ } + for _, byItem := range f.OrderByItems { + if _, isCnst := byItem.Expr.(*expression.Constant); isCnst { + continue + } + projExprs = append(projExprs, byItem.Expr) + newArg := &expression.Column{ + UniqueID: aggPlan.SCtx().GetSessionVars().AllocPlanColumnID(), + RetType: byItem.Expr.GetType(), + Index: cursor, + } + projSchemaCols = append(projSchemaCols, newArg) + byItem.Expr = newArg + cursor++ + } } for i, item := range groupByItems { @@ -146,7 +165,11 @@ func injectProjBelowAgg(aggPlan PhysicalPlan, aggFuncs []*aggregation.AggFuncDes // PhysicalTopN, some extra columns will be added into the schema of the // Projection, thus we need to add another Projection upon them to prune the // redundant columns. +<<<<<<< HEAD func injectProjBelowSort(p PhysicalPlan, orderByItems []*ByItems) PhysicalPlan { +======= +func InjectProjBelowSort(p PhysicalPlan, orderByItems []*util.ByItems) PhysicalPlan { +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) hasScalarFunc, numOrderByItems := false, len(orderByItems) for i := 0; !hasScalarFunc && i < numOrderByItems; i++ { _, isScalarFunc := orderByItems[i].Expr.(*expression.ScalarFunction) @@ -210,3 +233,67 @@ func injectProjBelowSort(p PhysicalPlan, orderByItems []*ByItems) PhysicalPlan { return topProj } +<<<<<<< HEAD +======= + +// TurnNominalSortIntoProj will turn nominal sort into two projections. This is to check if the scalar functions will +// overflow. +func TurnNominalSortIntoProj(p PhysicalPlan, onlyColumn bool, orderByItems []*util.ByItems) PhysicalPlan { + if onlyColumn { + return p.Children()[0] + } + + numOrderByItems := len(orderByItems) + childPlan := p.Children()[0] + + bottomProjSchemaCols := make([]*expression.Column, 0, len(childPlan.Schema().Columns)+numOrderByItems) + bottomProjExprs := make([]expression.Expression, 0, len(childPlan.Schema().Columns)+numOrderByItems) + for _, col := range childPlan.Schema().Columns { + newCol := col.Clone().(*expression.Column) + newCol.Index = childPlan.Schema().ColumnIndex(newCol) + bottomProjSchemaCols = append(bottomProjSchemaCols, newCol) + bottomProjExprs = append(bottomProjExprs, newCol) + } + + for _, item := range orderByItems { + itemExpr := item.Expr + if _, isScalarFunc := itemExpr.(*expression.ScalarFunction); !isScalarFunc { + continue + } + bottomProjExprs = append(bottomProjExprs, itemExpr) + newArg := &expression.Column{ + UniqueID: p.SCtx().GetSessionVars().AllocPlanColumnID(), + RetType: itemExpr.GetType(), + Index: len(bottomProjSchemaCols), + } + bottomProjSchemaCols = append(bottomProjSchemaCols, newArg) + } + + childProp := p.GetChildReqProps(0).Clone() + bottomProj := PhysicalProjection{ + Exprs: bottomProjExprs, + AvoidColumnEvaluator: false, + }.Init(p.SCtx(), childPlan.statsInfo().ScaleByExpectCnt(childProp.ExpectedCnt), p.SelectBlockOffset(), childProp) + bottomProj.SetSchema(expression.NewSchema(bottomProjSchemaCols...)) + bottomProj.SetChildren(childPlan) + + topProjExprs := make([]expression.Expression, 0, childPlan.Schema().Len()) + for i := range childPlan.Schema().Columns { + col := childPlan.Schema().Columns[i].Clone().(*expression.Column) + col.Index = i + topProjExprs = append(topProjExprs, col) + } + topProj := PhysicalProjection{ + Exprs: topProjExprs, + AvoidColumnEvaluator: false, + }.Init(p.SCtx(), childPlan.statsInfo().ScaleByExpectCnt(childProp.ExpectedCnt), p.SelectBlockOffset(), childProp) + topProj.SetSchema(childPlan.Schema().Clone()) + topProj.SetChildren(bottomProj) + + if origChildProj, isChildProj := childPlan.(*PhysicalProjection); isChildProj { + refine4NeighbourProj(bottomProj, origChildProj) + } + + return topProj +} +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) diff --git a/planner/core/rule_max_min_eliminate.go b/planner/core/rule_max_min_eliminate.go index 8ca1774d3e0ff..d771639f61592 100644 --- a/planner/core/rule_max_min_eliminate.go +++ b/planner/core/rule_max_min_eliminate.go @@ -179,8 +179,13 @@ func (a *maxMinEliminator) eliminateSingleMaxMin(agg *LogicalAggregation) *Logic // For max function, the sort order should be desc. desc := f.Name == ast.AggFuncMax // Compose Sort operator. +<<<<<<< HEAD sort := LogicalSort{}.Init(ctx) sort.ByItems = append(sort.ByItems, &ByItems{f.Args[0], desc}) +======= + sort := LogicalSort{}.Init(ctx, agg.blockOffset) + sort.ByItems = append(sort.ByItems, &util.ByItems{Expr: f.Args[0], Desc: desc}) +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) sort.SetChildren(child) child = sort } diff --git a/planner/core/rule_topn_push_down.go b/planner/core/rule_topn_push_down.go index 1db0820ab77cc..b6b0d8f45c45c 100644 --- a/planner/core/rule_topn_push_down.go +++ b/planner/core/rule_topn_push_down.go @@ -18,6 +18,7 @@ import ( "github.com/cznic/mathutil" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/planner/util" ) // pushDownTopNOptimizer pushes down the topN or limit. In the future we will remove the limit from `requiredProperty` in CBO phase. @@ -95,7 +96,7 @@ func (p *LogicalUnionAll) pushDownTopN(topN *LogicalTopN) LogicalPlan { if topN != nil { newTopN = LogicalTopN{Count: topN.Count + topN.Offset}.Init(p.ctx) for _, by := range topN.ByItems { - newTopN.ByItems = append(newTopN.ByItems, &ByItems{by.Expr, by.Desc}) + newTopN.ByItems = append(newTopN.ByItems, &util.ByItems{Expr: by.Expr, Desc: by.Desc}) } } p.children[i] = child.pushDownTopN(newTopN) @@ -141,8 +142,13 @@ func (p *LogicalJoin) pushDownTopNToChild(topN *LogicalTopN, idx int) LogicalPla newTopN := LogicalTopN{ Count: topN.Count + topN.Offset, +<<<<<<< HEAD ByItems: make([]*ByItems, len(topN.ByItems)), }.Init(topN.ctx) +======= + ByItems: make([]*util.ByItems, len(topN.ByItems)), + }.Init(topN.ctx, topN.blockOffset) +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) for i := range topN.ByItems { newTopN.ByItems[i] = topN.ByItems[i].Clone() } diff --git a/planner/core/task.go b/planner/core/task.go index 9a2258a7c3e9d..722655da06f40 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -22,6 +22,12 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" +<<<<<<< HEAD +======= + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/plancodec" @@ -358,8 +364,13 @@ func (p *NominalSort) attach2Task(tasks ...task) task { return tasks[0] } +<<<<<<< HEAD func (p *PhysicalTopN) getPushedDownTopN() *PhysicalTopN { newByItems := make([]*ByItems, 0, len(p.ByItems)) +======= +func (p *PhysicalTopN) getPushedDownTopN(childPlan PhysicalPlan) *PhysicalTopN { + newByItems := make([]*util.ByItems, 0, len(p.ByItems)) +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) for _, expr := range p.ByItems { newByItems = append(newByItems, expr.Clone()) } diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/testdata/plan_suite_in.json index a6aaf2907db64..eab01a994194a 100644 --- a/planner/core/testdata/plan_suite_in.json +++ b/planner/core/testdata/plan_suite_in.json @@ -429,5 +429,125 @@ "cases": [ "select t1.a, (select count(t2.a) from t t2 where t2.g in (select t3.d from t t3 where t3.c = t1.a)) as agg_col from t t1;" ] +<<<<<<< HEAD +======= + }, + { + "name": "TestIndexJoinHint", + "cases": [ + "select /*+ INL_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;", + "select /*+ INL_HASH_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;", + "select /*+ INL_MERGE_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;", + // Issue 15484 + "select /*+ inl_merge_join(t2) */ t1.a, t2.a from t t1 left join t t2 use index(g_2) on t1.g=t2.g", + "select /*+inl_merge_join(t2)*/ t1.a, t2.a from t t1 left join t t2 use index(g_2) on t1.g=t2.g order by t1.a" + ] + }, + { + "name": "TestAggToCopHint", + "cases": [ + "select /*+ AGG_TO_COP(), HASH_AGG(), USE_INDEX(t) */ sum(a) from ta group by a", + "select /*+ AGG_TO_COP(), USE_INDEX(t) */ sum(b) from ta group by b", + "select /*+ AGG_TO_COP(), HASH_AGG(), USE_INDEX(t) */ distinct a from ta group by a", + "select /*+ AGG_TO_COP(), HASH_AGG(), HASH_JOIN(t1), USE_INDEX(t1), USE_INDEX(t2) */ sum(t1.a) from ta t1, ta t2 where t1.a = t2.b group by t1.a" + ] + }, + { + "name": "TestPushdownDistinctEnable", + "cases": [ + "select /*+ HASH_AGG() */ avg(distinct a) from t;", // InjectProjBelowAgg + "select /*+ HASH_AGG() */ a, count(distinct a) from t;", // firstrow(a) cannot be removed. + "select /*+ HASH_AGG() */ avg(b), c, avg(b), count(distinct A, B), count(distinct A), count(distinct c), sum(b) from t group by c;", + "select /*+ STREAM_AGG() */ count(distinct c) from t group by c;", // can push down + "select /*+ STREAM_AGG() */ count(distinct c) from t;", // can not push down because c is not in group by + "select /*+ HASH_AGG() */ count(distinct c) from t;", // can push down + "select count(distinct c) from t group by c;", + "select count(distinct c) from t;", // should not use streamAgg because c is not in group by + + "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", + "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;" + ] + }, + { + "name": "TestPushdownDistinctDisable", + "cases": [ + // do not pushdown even AGG_TO_COP is specified. + "select /*+ HASH_AGG(), AGG_TO_COP() */ avg(distinct a) from t;", + "select /*+ HASH_AGG(), AGG_TO_COP() */ a, count(distinct a) from t;", + "select /*+ HASH_AGG(), AGG_TO_COP() */ avg(b), c, avg(b), count(distinct A, B), count(distinct A), count(distinct c), sum(b) from t group by c;", + "select /*+ STREAM_AGG(), AGG_TO_COP() */ count(distinct c) from t group by c;", + "select /*+ STREAM_AGG(), AGG_TO_COP() */ count(distinct c) from t;", + "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct c) from t;", + "select /*+ AGG_TO_COP() */ count(distinct c) from t group by c;", + + "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", + "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;" + ] + }, + { + "name": "TestPushdownDistinctEnableAggPushDownDisable", + "cases": [ + "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", + "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;" + ] + }, + { + "name": "TestGroupConcatOrderby", + "cases": [ + "select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from test;", + "select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from ptest;", + "select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from test;", + "select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from ptest;" + ] + }, + { + "name": "TestDAGPlanBuilderWindow", + "cases":[ + "select lead(a, 1) over (partition by null) as c from t" + ] + }, + { + "name": "TestDAGPlanBuilderWindowParallel", + "cases":[ + "select lead(a, 1) over (partition by null) as c from t", + "select lead(a, 1) over (partition by b) as c from t" + ] + }, + { + "name": "TestNominalSort", + "cases": [ + "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by t1.a", + "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by t1.a+1", + "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by t1.a-1", + "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by -t1.a", + "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by -t1.a+3", + "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1+t1.a", + "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1-t1.a", + "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1-t1.a+3", + "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1+t1.a+3", + "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 3*t1.a" + ] + }, + { + "name": "TestInlineProjection", + "cases":[ + "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1, t2 where t1.a = t2.a;", + "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", + "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1 left outer join t2 on t1.a = t2.a;", + "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1 right outer join t2 on t1.a = t2.a;", + "select 1 from (select /*+ HASH_JOIN(t1) */ t1.a in (select t2.a from t2) from t1) x;", + "select 1 from (select /*+ HASH_JOIN(t1) */ t1.a not in (select t2.a from t2) from t1) x;", + "select /*+ INL_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", + "select /*+ INL_HASH_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", + "select /*+ INL_MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", + "select /*+ MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;" + ] + }, + { + "name": "TestHintFromDiffDatabase", + "cases": [ + "select /*+ inl_hash_join(test.t1) */ * from test.t2 join test.t1 on test.t2.a = test.t1.a" + ] +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } ] diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index bf60c2f418f5c..675c559cf2a92 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -1010,7 +1010,689 @@ "Cases": [ { "SQL": "select t1.a, (select count(t2.a) from t t2 where t2.g in (select t3.d from t t3 where t3.c = t1.a)) as agg_col from t t1;", +<<<<<<< HEAD "Best": "Apply{TableReader(Table(t))->IndexJoin{IndexReader(Index(t.c_d_e)[[NULL,+inf]]->HashAgg)->HashAgg->IndexReader(Index(t.g)[[NULL,+inf]])}(test.t3.d,test.t2.g)}->StreamAgg" +======= + "Best": "Apply{IndexReader(Index(t.f)[[NULL,+inf]])->IndexMergeJoin{IndexReader(Index(t.c_d_e)[[NULL,+inf]]->HashAgg)->HashAgg->IndexReader(Index(t.g)[[NULL,+inf]])}(test.t.d,test.t.g)}->HashAgg" + } + ] + }, + { + "Name": "TestIndexJoinHint", + "Cases": [ + { + "SQL": "select /*+ INL_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;", + "Plan": "IndexJoin{IndexLookUp(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]), Table(t1))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)" + }, + { + "SQL": "select /*+ INL_HASH_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;", + "Plan": "IndexHashJoin{IndexLookUp(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]), Table(t1))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)" + }, + { + "SQL": "select /*+ INL_MERGE_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;", + "Plan": "IndexMergeJoin{IndexLookUp(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]), Table(t1))->Projection->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)" + }, + { + "SQL": "select /*+ inl_merge_join(t2) */ t1.a, t2.a from t t1 left join t t2 use index(g_2) on t1.g=t2.g", + "Plan": "IndexMergeJoin{IndexReader(Index(t.g_3)[[NULL,+inf]])->IndexReader(Index(t.g_2)[[NULL,+inf]]->Sel([not(isnull(test.t.g))]))}(test.t.g,test.t.g)" + }, + { + "SQL": "select /*+inl_merge_join(t2)*/ t1.a, t2.a from t t1 left join t t2 use index(g_2) on t1.g=t2.g order by t1.a", + "Plan": "IndexMergeJoin{IndexReader(Index(t.g_3)[[NULL,+inf]])->IndexReader(Index(t.g_2)[[NULL,+inf]]->Sel([not(isnull(test.t.g))]))}(test.t.g,test.t.g)->Sort" + } + ] + }, + { + "Name": "TestAggToCopHint", + "Cases": [ + { + "SQL": "select /*+ AGG_TO_COP(), HASH_AGG(), USE_INDEX(t) */ sum(a) from ta group by a", + "Best": "IndexReader(Index(ta.a)[[NULL,+inf]]->HashAgg)->HashAgg", + "Warning": "[planner:1815]use_index(test.t) is inapplicable, check whether the table(test.t) exists" + }, + { + "SQL": "select /*+ AGG_TO_COP(), USE_INDEX(t) */ sum(b) from ta group by b", + "Best": "TableReader(Table(ta)->HashAgg)->HashAgg", + "Warning": "[planner:1815]use_index(test.t) is inapplicable, check whether the table(test.t) exists" + }, + { + "SQL": "select /*+ AGG_TO_COP(), HASH_AGG(), USE_INDEX(t) */ distinct a from ta group by a", + "Best": "IndexReader(Index(ta.a)[[NULL,+inf]]->HashAgg)->HashAgg", + "Warning": "[planner:1815]use_index(test.t) is inapplicable, check whether the table(test.t) exists" + }, + { + "SQL": "select /*+ AGG_TO_COP(), HASH_AGG(), HASH_JOIN(t1), USE_INDEX(t1), USE_INDEX(t2) */ sum(t1.a) from ta t1, ta t2 where t1.a = t2.b group by t1.a", + "Best": "LeftHashJoin{TableReader(Table(ta)->Sel([not(isnull(test.ta.a))]))->TableReader(Table(ta)->Sel([not(isnull(test.ta.b))]))}(test.ta.a,test.ta.b)->Projection->HashAgg", + "Warning": "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + } + ] + }, + { + "Name": "TestPushdownDistinctEnable", + "Cases": [ + { + "SQL": "select /*+ HASH_AGG() */ avg(distinct a) from t;", + "Plan": [ + "HashAgg_8 1.00 root funcs:avg(distinct Column#6)->Column#5", + "└─Projection_10 1.00 root cast(test.t.a, decimal(65,4) BINARY)->Column#6", + " └─TableReader_9 1.00 root data:HashAgg_5", + " └─HashAgg_5 1.00 cop[tikv] group by:test.t.a, ", + " └─TableFullScan_7 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Result": [ + "1.5000" + ] + }, + { + "SQL": "select /*+ HASH_AGG() */ a, count(distinct a) from t;", + "Plan": [ + "Projection_4 1.00 root test.t.a, Column#5", + "└─HashAgg_8 1.00 root funcs:count(distinct test.t.a)->Column#5, funcs:firstrow(Column#6)->test.t.a", + " └─TableReader_9 1.00 root data:HashAgg_5", + " └─HashAgg_5 1.00 cop[tikv] group by:test.t.a, funcs:firstrow(test.t.a)->Column#6", + " └─TableFullScan_7 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Result": [ + "1 2" + ] + }, + { + "SQL": "select /*+ HASH_AGG() */ avg(b), c, avg(b), count(distinct A, B), count(distinct A), count(distinct c), sum(b) from t group by c;", + "Plan": [ + "Projection_4 8000.00 root Column#5, test.t.c, Column#5, Column#6, Column#7, Column#8, Column#9", + "└─HashAgg_8 8000.00 root group by:test.t.c, funcs:avg(Column#10, Column#11)->Column#5, funcs:count(distinct test.t.a, test.t.b)->Column#6, funcs:count(distinct test.t.a)->Column#7, funcs:count(distinct test.t.c)->Column#8, funcs:sum(Column#12)->Column#9, funcs:firstrow(test.t.c)->test.t.c", + " └─TableReader_9 8000.00 root data:HashAgg_5", + " └─HashAgg_5 8000.00 cop[tikv] group by:test.t.a, test.t.b, test.t.c, funcs:count(test.t.b)->Column#10, funcs:sum(test.t.b)->Column#11, funcs:sum(test.t.b)->Column#12", + " └─TableFullScan_7 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Result": [ + "1.0000 1 1.0000 1 1 1 1", + "1.3333 3 1.3333 3 2 1 4", + "2.0000 2.0000 1 1 0 2" + ] + }, + { + "SQL": "select /*+ STREAM_AGG() */ count(distinct c) from t group by c;", + "Plan": [ + "StreamAgg_11 8000.00 root group by:test.t.c, funcs:count(distinct test.t.c)->Column#5", + "└─IndexReader_12 8000.00 root index:StreamAgg_7", + " └─StreamAgg_7 8000.00 cop[tikv] group by:test.t.c, ", + " └─IndexFullScan_10 10000.00 cop[tikv] table:t, index:c(c) keep order:true, stats:pseudo" + ], + "Result": [ + "0", + "1", + "1" + ] + }, + { + "SQL": "select /*+ STREAM_AGG() */ count(distinct c) from t;", + "Plan": [ + "StreamAgg_7 1.00 root funcs:count(distinct test.t.c)->Column#5", + "└─IndexReader_11 10000.00 root index:IndexFullScan_10", + " └─IndexFullScan_10 10000.00 cop[tikv] table:t, index:c(c) keep order:false, stats:pseudo" + ], + "Result": [ + "2" + ] + }, + { + "SQL": "select /*+ HASH_AGG() */ count(distinct c) from t;", + "Plan": [ + "HashAgg_9 1.00 root funcs:count(distinct test.t.c)->Column#5", + "└─IndexReader_10 1.00 root index:HashAgg_5", + " └─HashAgg_5 1.00 cop[tikv] group by:test.t.c, ", + " └─IndexFullScan_8 10000.00 cop[tikv] table:t, index:c(c) keep order:false, stats:pseudo" + ], + "Result": [ + "2" + ] + }, + { + "SQL": "select count(distinct c) from t group by c;", + "Plan": [ + "HashAgg_10 8000.00 root group by:test.t.c, funcs:count(distinct test.t.c)->Column#5", + "└─IndexReader_11 8000.00 root index:HashAgg_5", + " └─HashAgg_5 8000.00 cop[tikv] group by:test.t.c, ", + " └─IndexFullScan_9 10000.00 cop[tikv] table:t, index:c(c) keep order:false, stats:pseudo" + ], + "Result": [ + "0", + "1", + "1" + ] + }, + { + "SQL": "select count(distinct c) from t;", + "Plan": [ + "HashAgg_9 1.00 root funcs:count(distinct test.t.c)->Column#5", + "└─IndexReader_10 1.00 root index:HashAgg_5", + " └─HashAgg_5 1.00 cop[tikv] group by:test.t.c, ", + " └─IndexFullScan_8 10000.00 cop[tikv] table:t, index:c(c) keep order:false, stats:pseudo" + ], + "Result": [ + "2" + ] + }, + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", + "Plan": [ + "HashAgg_11 1.00 root funcs:sum(distinct Column#7)->Column#4", + "└─Projection_23 16000.00 root cast(test.pt.b, decimal(65,0) BINARY)->Column#7", + " └─Union_12 16000.00 root ", + " ├─HashAgg_16 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", + " │ └─TableReader_17 8000.00 root data:HashAgg_13", + " │ └─HashAgg_13 8000.00 cop[tikv] group by:test.pt.b, ", + " │ └─TableFullScan_15 10000.00 cop[tikv] table:pt, partition:p0 keep order:false, stats:pseudo", + " └─HashAgg_21 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", + " └─TableReader_22 8000.00 root data:HashAgg_18", + " └─HashAgg_18 8000.00 cop[tikv] group by:test.pt.b, ", + " └─TableFullScan_20 10000.00 cop[tikv] table:pt, partition:p1 keep order:false, stats:pseudo" + ], + "Result": [ + "" + ] + }, + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;", + "Plan": [ + "HashAgg_14 1.00 root funcs:count(distinct Column#5)->Column#6", + "└─Union_15 16000.00 root ", + " ├─HashAgg_19 8000.00 root group by:test.ta.a, funcs:firstrow(test.ta.a)->Column#5", + " │ └─TableReader_20 8000.00 root data:HashAgg_16", + " │ └─HashAgg_16 8000.00 cop[tikv] group by:test.ta.a, ", + " │ └─TableFullScan_18 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", + " └─HashAgg_24 8000.00 root group by:test.tb.a, funcs:firstrow(test.tb.a)->Column#5", + " └─TableReader_25 8000.00 root data:HashAgg_21", + " └─HashAgg_21 8000.00 cop[tikv] group by:test.tb.a, ", + " └─TableFullScan_23 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo" + ], + "Result": [ + "1" + ] + } + ] + }, + { + "Name": "TestPushdownDistinctDisable", + "Cases": [ + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ avg(distinct a) from t;", + "Plan": [ + "HashAgg_5 1.00 root funcs:avg(distinct Column#6)->Column#5", + "└─Projection_8 10000.00 root cast(test.t.a, decimal(65,4) BINARY)->Column#6", + " └─TableReader_7 10000.00 root data:TableFullScan_6", + " └─TableFullScan_6 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Result": [ + "1.5000" + ] + }, + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ a, count(distinct a) from t;", + "Plan": [ + "Projection_4 1.00 root test.t.a, Column#5", + "└─HashAgg_5 1.00 root funcs:count(distinct test.t.a)->Column#5, funcs:firstrow(test.t.a)->test.t.a", + " └─TableReader_7 10000.00 root data:TableFullScan_6", + " └─TableFullScan_6 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Result": [ + "1 2" + ] + }, + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ avg(b), c, avg(b), count(distinct A, B), count(distinct A), count(distinct c), sum(b) from t group by c;", + "Plan": [ + "Projection_4 8000.00 root Column#5, test.t.c, Column#5, Column#6, Column#7, Column#8, Column#9", + "└─HashAgg_5 8000.00 root group by:Column#17, funcs:avg(Column#10)->Column#5, funcs:count(distinct Column#11, Column#12)->Column#6, funcs:count(distinct Column#13)->Column#7, funcs:count(distinct Column#14)->Column#8, funcs:sum(Column#15)->Column#9, funcs:firstrow(Column#16)->test.t.c", + " └─Projection_8 10000.00 root cast(test.t.b, decimal(65,4) BINARY)->Column#10, test.t.a, test.t.b, test.t.a, test.t.c, cast(test.t.b, decimal(65,0) BINARY)->Column#15, test.t.c, test.t.c", + " └─TableReader_7 10000.00 root data:TableFullScan_6", + " └─TableFullScan_6 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Result": [ + "1.0000 1 1.0000 1 1 1 1", + "1.3333 3 1.3333 3 2 1 4", + "2.0000 2.0000 1 1 0 2" + ] + }, + { + "SQL": "select /*+ STREAM_AGG(), AGG_TO_COP() */ count(distinct c) from t group by c;", + "Plan": [ + "StreamAgg_6 8000.00 root group by:test.t.c, funcs:count(distinct test.t.c)->Column#5", + "└─IndexReader_9 10000.00 root index:IndexFullScan_8", + " └─IndexFullScan_8 10000.00 cop[tikv] table:t, index:c(c) keep order:true, stats:pseudo" + ], + "Result": [ + "0", + "1", + "1" + ] + }, + { + "SQL": "select /*+ STREAM_AGG(), AGG_TO_COP() */ count(distinct c) from t;", + "Plan": [ + "StreamAgg_6 1.00 root funcs:count(distinct test.t.c)->Column#5", + "└─IndexReader_11 10000.00 root index:IndexFullScan_10", + " └─IndexFullScan_10 10000.00 cop[tikv] table:t, index:c(c) keep order:false, stats:pseudo" + ], + "Result": [ + "2" + ] + }, + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct c) from t;", + "Plan": [ + "HashAgg_5 1.00 root funcs:count(distinct test.t.c)->Column#5", + "└─IndexReader_9 10000.00 root index:IndexFullScan_8", + " └─IndexFullScan_8 10000.00 cop[tikv] table:t, index:c(c) keep order:false, stats:pseudo" + ], + "Result": [ + "2" + ] + }, + { + "SQL": "select /*+ AGG_TO_COP() */ count(distinct c) from t group by c;", + "Plan": [ + "StreamAgg_6 8000.00 root group by:test.t.c, funcs:count(distinct test.t.c)->Column#5", + "└─IndexReader_12 10000.00 root index:IndexFullScan_11", + " └─IndexFullScan_11 10000.00 cop[tikv] table:t, index:c(c) keep order:true, stats:pseudo" + ], + "Result": [ + "0", + "1", + "1" + ] + }, + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", + "Plan": [ + "HashAgg_11 1.00 root funcs:sum(distinct Column#7)->Column#4", + "└─Projection_23 16000.00 root cast(test.pt.b, decimal(65,0) BINARY)->Column#7", + " └─Union_12 16000.00 root ", + " ├─HashAgg_16 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", + " │ └─TableReader_17 8000.00 root data:HashAgg_13", + " │ └─HashAgg_13 8000.00 cop[tikv] group by:test.pt.b, ", + " │ └─TableFullScan_15 10000.00 cop[tikv] table:pt, partition:p0 keep order:false, stats:pseudo", + " └─HashAgg_21 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", + " └─TableReader_22 8000.00 root data:HashAgg_18", + " └─HashAgg_18 8000.00 cop[tikv] group by:test.pt.b, ", + " └─TableFullScan_20 10000.00 cop[tikv] table:pt, partition:p1 keep order:false, stats:pseudo" + ], + "Result": [ + "" + ] + }, + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;", + "Plan": [ + "HashAgg_14 1.00 root funcs:count(distinct Column#5)->Column#6", + "└─Union_15 16000.00 root ", + " ├─HashAgg_19 8000.00 root group by:test.ta.a, funcs:firstrow(test.ta.a)->Column#5", + " │ └─TableReader_20 8000.00 root data:HashAgg_16", + " │ └─HashAgg_16 8000.00 cop[tikv] group by:test.ta.a, ", + " │ └─TableFullScan_18 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", + " └─HashAgg_24 8000.00 root group by:test.tb.a, funcs:firstrow(test.tb.a)->Column#5", + " └─TableReader_25 8000.00 root data:HashAgg_21", + " └─HashAgg_21 8000.00 cop[tikv] group by:test.tb.a, ", + " └─TableFullScan_23 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo" + ], + "Result": [ + "1" + ] + } + ] + }, + { + "Name": "TestPushdownDistinctEnableAggPushDownDisable", + "Cases": [ + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", + "Plan": [ + "HashAgg_8 1.00 root funcs:sum(distinct Column#5)->Column#4", + "└─Projection_14 20000.00 root cast(test.pt.b, decimal(65,0) BINARY)->Column#5", + " └─Union_9 20000.00 root ", + " ├─TableReader_11 10000.00 root data:TableFullScan_10", + " │ └─TableFullScan_10 10000.00 cop[tikv] table:pt, partition:p0 keep order:false, stats:pseudo", + " └─TableReader_13 10000.00 root data:TableFullScan_12", + " └─TableFullScan_12 10000.00 cop[tikv] table:pt, partition:p1 keep order:false, stats:pseudo" + ], + "Result": [ + "" + ] + }, + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;", + "Plan": [ + "HashAgg_11 1.00 root funcs:count(distinct Column#5)->Column#6", + "└─Union_12 20000.00 root ", + " ├─TableReader_15 10000.00 root data:TableFullScan_14", + " │ └─TableFullScan_14 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", + " └─TableReader_18 10000.00 root data:TableFullScan_17", + " └─TableFullScan_17 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo" + ], + "Result": [ + "1" + ] + } + ] + }, + { + "Name": "TestGroupConcatOrderby", + "Cases": [ + { + "SQL": "select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from test;", + "Plan": [ + "HashAgg_5 1.00 root funcs:group_concat(Column#6 order by Column#7 desc separator \"++\")->Column#4, funcs:group_concat(Column#8 order by Column#9 desc, Column#10 asc separator \"--\")->Column#5", + "└─Projection_18 10000.00 root cast(test.test.name, var_string(20))->Column#6, test.test.name, cast(test.test.id, var_string(20))->Column#8, test.test.name, test.test.id", + " └─TableReader_11 10000.00 root data:TableFullScan_10", + " └─TableFullScan_10 10000.00 cop[tikv] table:test keep order:false, stats:pseudo" + ], + "Result": [ + "500++200++30++20++20++10 3--3--1--1--2--1" + ] + }, + { + "SQL": "select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from ptest;", + "Plan": [ + "HashAgg_10 1.00 root funcs:group_concat(Column#6 order by Column#7 desc separator \"++\")->Column#4, funcs:group_concat(Column#8 order by Column#9 desc, Column#10 asc separator \"--\")->Column#5", + "└─Projection_23 20000.00 root cast(test.ptest.name, var_string(20))->Column#6, test.ptest.name, cast(test.ptest.id, var_string(20))->Column#8, test.ptest.name, test.ptest.id", + " └─Union_13 20000.00 root ", + " ├─TableReader_15 10000.00 root data:TableFullScan_14", + " │ └─TableFullScan_14 10000.00 cop[tikv] table:ptest, partition:p0 keep order:false, stats:pseudo", + " └─TableReader_17 10000.00 root data:TableFullScan_16", + " └─TableFullScan_16 10000.00 cop[tikv] table:ptest, partition:p1 keep order:false, stats:pseudo" + ], + "Result": [ + "500++200++30++20++20++10 3--3--1--1--2--1" + ] + }, + { + "SQL": "select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from test;", + "Plan": [ + "HashAgg_5 1.00 root funcs:group_concat(distinct Column#5 order by Column#6 desc separator \",\")->Column#4", + "└─Projection_9 10000.00 root cast(test.test.name, var_string(20))->Column#5, test.test.name", + " └─TableReader_8 10000.00 root data:TableFullScan_7", + " └─TableFullScan_7 10000.00 cop[tikv] table:test keep order:false, stats:pseudo" + ], + "Result": [ + "500,200,30,20,10" + ] + }, + { + "SQL": "select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from ptest;", + "Plan": [ + "StreamAgg_9 1.00 root funcs:group_concat(distinct Column#5 order by Column#6 desc separator \",\")->Column#4", + "└─Projection_20 20000.00 root cast(test.ptest.name, var_string(20))->Column#5, test.ptest.name", + " └─Union_15 20000.00 root ", + " ├─TableReader_17 10000.00 root data:TableFullScan_16", + " │ └─TableFullScan_16 10000.00 cop[tikv] table:ptest, partition:p0 keep order:false, stats:pseudo", + " └─TableReader_19 10000.00 root data:TableFullScan_18", + " └─TableFullScan_18 10000.00 cop[tikv] table:ptest, partition:p1 keep order:false, stats:pseudo" + ], + "Result": [ + "500,200,30,20,10" + ] + } + ] + }, + { + "Name": "TestDAGPlanBuilderWindow", + "Cases": [ + { + "SQL": "select lead(a, 1) over (partition by null) as c from t", + "Best": "IndexReader(Index(t.f)[[NULL,+inf]])->Window(lead(test.t.a, 1)->Column#14 over())->Projection" + } + ] + }, + { + "Name": "TestDAGPlanBuilderWindowParallel", + "Cases": [ + { + "SQL": "select lead(a, 1) over (partition by null) as c from t", + "Best": "IndexReader(Index(t.f)[[NULL,+inf]])->Window(lead(test.t.a, 1)->Column#14 over())->Projection" + }, + { + "SQL": "select lead(a, 1) over (partition by b) as c from t", + "Best": "TableReader(Table(t))->Sort->Window(lead(test.t.a, 1)->Column#14 over(partition by test.t.b))->Partition(execution info: concurrency:4, data source:TableReader_10)->Projection" + } + ] + }, + { + "Name": "TestNominalSort", + "Cases": [ + { + "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by t1.a", + "Plan": [ + "MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + "├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", + "│ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", + "└─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", + " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" + ], + "Result": [ + "1", + "1", + "2" + ] + }, + { + "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by t1.a+1", + "Plan": [ + "Projection_24 12487.50 root test.t.a", + "└─Projection_23 12487.50 root test.t.a, plus(test.t.a, 1)->Column#7", + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", + " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", + " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", + " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" + ], + "Result": [ + "1", + "1", + "2" + ] + }, + { + "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by t1.a-1", + "Plan": [ + "Projection_24 12487.50 root test.t.a", + "└─Projection_23 12487.50 root test.t.a, minus(test.t.a, 1)->Column#7", + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", + " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", + " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", + " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" + ], + "Result": [ + "1", + "1", + "2" + ] + }, + { + "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by -t1.a", + "Plan": [ + "Projection_32 12487.50 root test.t.a", + "└─Projection_31 12487.50 root test.t.a, unaryminus(test.t.a)->Column#7", + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", + " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", + " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", + " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" + ], + "Result": [ + "2", + "1", + "1" + ] + }, + { + "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by -t1.a+3", + "Plan": [ + "Projection_32 12487.50 root test.t.a", + "└─Projection_31 12487.50 root test.t.a, plus(unaryminus(test.t.a), 3)->Column#7", + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", + " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", + " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", + " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" + ], + "Result": [ + "2", + "1", + "1" + ] + }, + { + "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1+t1.a", + "Plan": [ + "Projection_24 12487.50 root test.t.a", + "└─Projection_23 12487.50 root test.t.a, plus(1, test.t.a)->Column#7", + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", + " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", + " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", + " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" + ], + "Result": [ + "1", + "1", + "2" + ] + }, + { + "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1-t1.a", + "Plan": [ + "Projection_32 12487.50 root test.t.a", + "└─Projection_31 12487.50 root test.t.a, minus(1, test.t.a)->Column#7", + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", + " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", + " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", + " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" + ], + "Result": [ + "2", + "1", + "1" + ] + }, + { + "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1-t1.a+3", + "Plan": [ + "Projection_32 12487.50 root test.t.a", + "└─Projection_31 12487.50 root test.t.a, plus(minus(1, test.t.a), 3)->Column#7", + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", + " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", + " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", + " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" + ], + "Result": [ + "2", + "1", + "1" + ] + }, + { + "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1+t1.a+3", + "Plan": [ + "Projection_24 12487.50 root test.t.a", + "└─Projection_23 12487.50 root test.t.a, plus(plus(1, test.t.a), 3)->Column#7", + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", + " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", + " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", + " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" + ], + "Result": [ + "1", + "1", + "2" + ] + }, + { + "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 3*t1.a", + "Plan": [ + "Projection_19 12487.50 root test.t.a", + "└─Sort_7 12487.50 root Column#7:asc", + " └─Projection_20 12487.50 root test.t.a, mul(3, test.t.a)->Column#7", + " └─MergeJoin_9 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_14(Build) 9990.00 root index:IndexFullScan_13", + " │ └─IndexFullScan_13 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", + " └─IndexReader_12(Probe) 9990.00 root index:IndexFullScan_11", + " └─IndexFullScan_11 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" + ], + "Result": [ + "1", + "1", + "2" + ] + } + ] + }, + { + "Name": "TestInlineProjection", + "Cases": [ + { + "SQL": "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1, t2 where t1.a = t2.a;", + "Plan": "LeftHashJoin{TableReader(Table(t1)->Sel([not(isnull(test.t1.a))]))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t1.a,test.t2.a)", + "Hints": "use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_1` `test`.`t2` ), hash_join(@`sel_1` `test`.`t1`)" + }, + { + "SQL": "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", + "Plan": "LeftHashJoin{TableReader(Table(t1)->Sel([not(isnull(test.t1.a))]))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t1.a,test.t2.a)", + "Hints": "use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_1` `test`.`t2` ), hash_join(@`sel_1` `test`.`t1`)" + }, + { + "SQL": "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1 left outer join t2 on t1.a = t2.a;", + "Plan": "LeftHashJoin{TableReader(Table(t1))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t1.a,test.t2.a)", + "Hints": "use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_1` `test`.`t2` ), hash_join(@`sel_1` `test`.`t1`)" + }, + { + "SQL": "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1 right outer join t2 on t1.a = t2.a;", + "Plan": "RightHashJoin{TableReader(Table(t1)->Sel([not(isnull(test.t1.a))]))->TableReader(Table(t2))}(test.t1.a,test.t2.a)", + "Hints": "use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_1` `test`.`t2` ), hash_join(@`sel_1` `test`.`t1`)" + }, + { + "SQL": "select 1 from (select /*+ HASH_JOIN(t1) */ t1.a in (select t2.a from t2) from t1) x;", + "Plan": "LeftHashJoin{IndexReader(Index(t1.idx_a)[[NULL,+inf]])->IndexReader(Index(t2.idx_a)[[NULL,+inf]])}->Projection", + "Hints": "use_index(@`sel_2` `test`.`t1` `idx_a`), use_index(@`sel_3` `test`.`t2` `idx_a`), hash_join(@`sel_2` `test`.`t1`)" + }, + { + "SQL": "select 1 from (select /*+ HASH_JOIN(t1) */ t1.a not in (select t2.a from t2) from t1) x;", + "Plan": "LeftHashJoin{IndexReader(Index(t1.idx_a)[[NULL,+inf]])->IndexReader(Index(t2.idx_a)[[NULL,+inf]])}->Projection", + "Hints": "use_index(@`sel_2` `test`.`t1` `idx_a`), use_index(@`sel_3` `test`.`t2` `idx_a`), hash_join(@`sel_2` `test`.`t1`)" + }, + { + "SQL": "select /*+ INL_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", + "Plan": "IndexJoin{IndexLookUp(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]), Table(t1))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)", + "Hints": "use_index(@`sel_1` `test`.`t1` `idx_a`), use_index(@`sel_1` `test`.`t2` ), inl_join(@`sel_1` `test`.`t1`)" + }, + { + "SQL": "select /*+ INL_HASH_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", + "Plan": "IndexHashJoin{IndexLookUp(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]), Table(t1))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)", + "Hints": "use_index(@`sel_1` `test`.`t1` `idx_a`), use_index(@`sel_1` `test`.`t2` ), inl_hash_join(@`sel_1` )" + }, + { + "SQL": "select /*+ INL_MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", + "Plan": "IndexMergeJoin{IndexLookUp(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]), Table(t1))->Projection->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)", + "Hints": "use_index(@`sel_1` `test`.`t1` `idx_a`), use_index(@`sel_1` `test`.`t2` )" + }, + { + "SQL": "select /*+ MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", + "Plan": "MergeInnerJoin{IndexLookUp(Index(t1.idx_a)[[-inf,+inf]], Table(t1))->Projection->IndexLookUp(Index(t2.idx_a)[[-inf,+inf]], Table(t2))->Projection}(test.t1.a,test.t2.a)", + "Hints": "use_index(@`sel_1` `test`.`t1` `idx_a`), use_index(@`sel_1` `test`.`t2` `idx_a`)" + } + ] + }, + { + "Name": "TestHintFromDiffDatabase", + "Cases": [ + { + "SQL": "select /*+ inl_hash_join(test.t1) */ * from test.t2 join test.t1 on test.t2.a = test.t1.a", + "Plan": "IndexHashJoin{IndexReader(Index(t2.idx_a)[[-inf,+inf]])->IndexReader(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]))}(test.t2.a,test.t1.a)" +>>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } ] } diff --git a/planner/util/byitem.go b/planner/util/byitem.go new file mode 100644 index 0000000000000..550bb93572cbe --- /dev/null +++ b/planner/util/byitem.go @@ -0,0 +1,45 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package util + +import ( + "fmt" + + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/sessionctx" +) + +// ByItems wraps a "by" item. +type ByItems struct { + Expr expression.Expression + Desc bool +} + +// String implements fmt.Stringer interface. +func (by *ByItems) String() string { + if by.Desc { + return fmt.Sprintf("%s true", by.Expr) + } + return by.Expr.String() +} + +// Clone makes a copy of ByItems. +func (by *ByItems) Clone() *ByItems { + return &ByItems{Expr: by.Expr.Clone(), Desc: by.Desc} +} + +// Equal checks whether two ByItems are equal. +func (by *ByItems) Equal(ctx sessionctx.Context, other *ByItems) bool { + return by.Expr.Equal(ctx, other.Expr) && by.Desc == other.Desc +} diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 26d32171e6af9..12e6fecac140c 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -22,6 +22,7 @@ import ( "sync/atomic" "time" + "github.com/cznic/mathutil" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" @@ -313,9 +314,15 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, } return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) case GroupConcatMaxLen: - // The reasonable range of 'group_concat_max_len' is 4~18446744073709551615(64-bit platforms) - // See https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_group_concat_max_len for details - return checkUInt64SystemVar(name, value, 4, math.MaxUint64, vars) + // https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_group_concat_max_len + // Minimum Value 4 + // Maximum Value (64-bit platforms) 18446744073709551615 + // Maximum Value (32-bit platforms) 4294967295 + maxLen := uint64(math.MaxUint64) + if mathutil.IntBits == 32 { + maxLen = uint64(math.MaxUint32) + } + return checkUInt64SystemVar(name, value, 4, maxLen, vars) case InteractiveTimeout: return checkUInt64SystemVar(name, value, 1, secondsPerYear, vars) case InnodbCommitConcurrency: From a2db2b1b3cd3c1029e84a269b07a702a1a3ab788 Mon Sep 17 00:00:00 2001 From: Feng Liyuan Date: Wed, 6 May 2020 22:57:50 +0800 Subject: [PATCH 2/9] fix conflict --- executor/aggfuncs/func_group_concat.go | 2 +- executor/aggregate_test.go | 4 - executor/benchmark_test.go | 989 -------- executor/builder.go | 17 - executor/executor_pkg_test.go | 94 - expression/aggregation/aggregation.go | 28 - planner/cascades/enforcer_rules.go | 9 +- planner/cascades/transformation_rules.go | 2199 ------------------ planner/core/exhaust_physical_plans.go | 5 - planner/core/explain.go | 206 -- planner/core/find_best_task.go | 33 +- planner/core/logical_plan_builder.go | 27 - planner/core/logical_plan_test.go | 14 - planner/core/logical_plans.go | 1 + planner/core/pb_to_plan.go | 277 --- planner/core/physical_plan_test.go | 757 ------ planner/core/physical_plans.go | 13 - planner/core/plan.go | 6 +- planner/core/planbuilder.go | 11 - planner/core/rule_aggregation_push_down.go | 14 +- planner/core/rule_column_pruning.go | 23 - planner/core/rule_inject_extra_projection.go | 70 +- planner/core/rule_max_min_eliminate.go | 8 +- planner/core/rule_topn_push_down.go | 7 +- planner/core/task.go | 10 - planner/core/testdata/plan_suite_in.json | 120 - planner/core/testdata/plan_suite_out.json | 682 ------ 27 files changed, 15 insertions(+), 5611 deletions(-) delete mode 100644 planner/core/pb_to_plan.go diff --git a/executor/aggfuncs/func_group_concat.go b/executor/aggfuncs/func_group_concat.go index b0612292921f6..932d25b901514 100644 --- a/executor/aggfuncs/func_group_concat.go +++ b/executor/aggfuncs/func_group_concat.go @@ -19,8 +19,8 @@ import ( "sort" "sync/atomic" + "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" - mysql "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index b3345a8b220c1..0d30a444f4518 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -399,12 +399,8 @@ func (s *testSuite1) TestAggPrune(c *C) { tk.MustQuery("SELECT a, MIN(b), MAX(b) FROM t GROUP BY a").Check(testkit.Rows("1 11 11", "3 ")) } -<<<<<<< HEAD func (s *testSuite1) TestGroupConcatAggr(c *C) { -======= -func (s *testSuiteAgg) TestGroupConcatAggr(c *C) { var err error ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) // issue #5411 tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index 3015868a9d4e2..aff3cd4e375a3 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/planner/property" - "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" @@ -376,30 +375,7 @@ func buildWindowExecutor(ctx sessionctx.Context, windowFunc string, src Executor desc, _ := aggregation.NewWindowFuncDesc(ctx, windowFunc, args) plan.WindowFuncDescs = []*aggregation.WindowFuncDesc{desc} for _, col := range partitionBy { -<<<<<<< HEAD plan.PartitionBy = append(plan.PartitionBy, property.Item{Col: col}) -======= - win.PartitionBy = append(win.PartitionBy, property.Item{Col: col}) - } - win.Frame = frame - win.OrderBy = nil - - win.SetSchema(winSchema) - win.Init(ctx, nil, 0) - - var tail core.PhysicalPlan = win - if !dataSourceSorted { - byItems := make([]*util.ByItems, 0, len(partitionBy)) - for _, col := range partitionBy { - byItems = append(byItems, &util.ByItems{Expr: col, Desc: false}) - } - sort := &core.PhysicalSort{ByItems: byItems} - sort.SetChildren(src) - win.SetChildren(sort) - tail = sort - } else { - win.SetChildren(src) ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } plan.OrderBy = nil plan.SetSchema(schema) @@ -527,968 +503,3 @@ func BenchmarkWindowFunctions(b *testing.B) { }) } } -<<<<<<< HEAD -======= - -func BenchmarkWindowFunctionsWithSlidingWindow(b *testing.B) { - baseBenchmarkWindowFunctionsWithSlidingWindow(b, ast.Rows) - baseBenchmarkWindowFunctionsWithSlidingWindow(b, ast.Ranges) -} - -type hashJoinTestCase struct { - rows int - cols []*types.FieldType - concurrency int - ctx sessionctx.Context - keyIdx []int - joinType core.JoinType - disk bool - useOuterToBuild bool - rawData string - childrenUsedSchema [][]bool -} - -func (tc hashJoinTestCase) columns() []*expression.Column { - ret := make([]*expression.Column, 0) - for i, t := range tc.cols { - column := &expression.Column{Index: i, RetType: t, UniqueID: tc.ctx.GetSessionVars().AllocPlanColumnID()} - ret = append(ret, column) - } - return ret -} - -func (tc hashJoinTestCase) String() string { - return fmt.Sprintf("(rows:%v, cols:%v, concurency:%v, joinKeyIdx: %v, disk:%v)", - tc.rows, tc.cols, tc.concurrency, tc.keyIdx, tc.disk) -} - -func defaultHashJoinTestCase(cols []*types.FieldType, joinType core.JoinType, useOuterToBuild bool) *hashJoinTestCase { - ctx := mock.NewContext() - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize - ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, -1) - ctx.GetSessionVars().StmtCtx.DiskTracker = disk.NewTracker(nil, -1) - ctx.GetSessionVars().IndexLookupJoinConcurrency = 4 - tc := &hashJoinTestCase{rows: 100000, concurrency: 4, ctx: ctx, keyIdx: []int{0, 1}, rawData: wideString} - tc.cols = cols - tc.useOuterToBuild = useOuterToBuild - tc.joinType = joinType - return tc -} - -func prepare4HashJoin(testCase *hashJoinTestCase, innerExec, outerExec Executor) *HashJoinExec { - if testCase.useOuterToBuild { - innerExec, outerExec = outerExec, innerExec - } - cols0 := innerExec.Schema().Columns - cols1 := outerExec.Schema().Columns - - joinSchema := expression.NewSchema() - if testCase.childrenUsedSchema != nil { - for i, used := range testCase.childrenUsedSchema[0] { - if used { - joinSchema.Append(cols0[i]) - } - } - for i, used := range testCase.childrenUsedSchema[1] { - if used { - joinSchema.Append(cols1[i]) - } - } - } else { - joinSchema.Append(cols0...) - joinSchema.Append(cols1...) - } - - joinKeys := make([]*expression.Column, 0, len(testCase.keyIdx)) - for _, keyIdx := range testCase.keyIdx { - joinKeys = append(joinKeys, cols0[keyIdx]) - } - probeKeys := make([]*expression.Column, 0, len(testCase.keyIdx)) - for _, keyIdx := range testCase.keyIdx { - probeKeys = append(probeKeys, cols1[keyIdx]) - } - e := &HashJoinExec{ - baseExecutor: newBaseExecutor(testCase.ctx, joinSchema, stringutil.StringerStr("HashJoin"), innerExec, outerExec), - concurrency: uint(testCase.concurrency), - joinType: testCase.joinType, // 0 for InnerJoin, 1 for LeftOutersJoin, 2 for RightOuterJoin - isOuterJoin: false, - buildKeys: joinKeys, - probeKeys: probeKeys, - buildSideExec: innerExec, - probeSideExec: outerExec, - buildSideEstCount: float64(testCase.rows), - useOuterToBuild: testCase.useOuterToBuild, - } - - childrenUsedSchema := markChildrenUsedCols(e.Schema(), e.children[0].Schema(), e.children[1].Schema()) - defaultValues := make([]types.Datum, e.buildSideExec.Schema().Len()) - lhsTypes, rhsTypes := retTypes(innerExec), retTypes(outerExec) - e.joiners = make([]joiner, e.concurrency) - for i := uint(0); i < e.concurrency; i++ { - e.joiners[i] = newJoiner(testCase.ctx, e.joinType, true, defaultValues, - nil, lhsTypes, rhsTypes, childrenUsedSchema) - } - memLimit := int64(-1) - if testCase.disk { - memLimit = 1 - } - t := memory.NewTracker(stringutil.StringerStr("root of prepare4HashJoin"), memLimit) - t.SetActionOnExceed(nil) - t2 := disk.NewTracker(stringutil.StringerStr("root of prepare4HashJoin"), -1) - e.ctx.GetSessionVars().StmtCtx.MemTracker = t - e.ctx.GetSessionVars().StmtCtx.DiskTracker = t2 - return e -} - -func benchmarkHashJoinExecWithCase(b *testing.B, casTest *hashJoinTestCase) { - opt1 := mockDataSourceParameters{ - rows: casTest.rows, - ctx: casTest.ctx, - genDataFunc: func(row int, typ *types.FieldType) interface{} { - switch typ.Tp { - case mysql.TypeLong, mysql.TypeLonglong: - return int64(row) - case mysql.TypeVarString: - return casTest.rawData - case mysql.TypeDouble: - return float64(row) - default: - panic("not implement") - } - }, - } - opt2 := opt1 - opt1.schema = expression.NewSchema(casTest.columns()...) - opt2.schema = expression.NewSchema(casTest.columns()...) - dataSource1 := buildMockDataSource(opt1) - dataSource2 := buildMockDataSource(opt2) - - b.ResetTimer() - for i := 0; i < b.N; i++ { - b.StopTimer() - exec := prepare4HashJoin(casTest, dataSource1, dataSource2) - tmpCtx := context.Background() - chk := newFirstChunk(exec) - dataSource1.prepareChunks() - dataSource2.prepareChunks() - - totalRow := 0 - b.StartTimer() - if err := exec.Open(tmpCtx); err != nil { - b.Fatal(err) - } - for { - if err := exec.Next(tmpCtx, chk); err != nil { - b.Fatal(err) - } - if chk.NumRows() == 0 { - break - } - totalRow += chk.NumRows() - } - - if spilled := exec.rowContainer.alreadySpilled(); spilled != casTest.disk { - b.Fatal("wrong usage with disk:", spilled, casTest.disk) - } - if err := exec.Close(); err != nil { - b.Fatal(err) - } - b.StopTimer() - if totalRow == 0 { - b.Fatal("totalRow == 0") - } - } -} - -func BenchmarkHashJoinInlineProjection(b *testing.B) { - cols := []*types.FieldType{ - types.NewFieldType(mysql.TypeLonglong), - types.NewFieldType(mysql.TypeVarString), - } - - b.ReportAllocs() - - { - cas := defaultHashJoinTestCase(cols, 0, false) - cas.keyIdx = []int{0} - cas.childrenUsedSchema = [][]bool{ - {false, true}, - {false, false}, - } - b.Run("InlineProjection:ON", func(b *testing.B) { - benchmarkHashJoinExecWithCase(b, cas) - }) - } - - { - cas := defaultHashJoinTestCase(cols, 0, false) - cas.keyIdx = []int{0} - b.Run("InlineProjection:OFF", func(b *testing.B) { - benchmarkHashJoinExecWithCase(b, cas) - }) - } -} - -func BenchmarkHashJoinExec(b *testing.B) { - lvl := log.GetLevel() - log.SetLevel(zapcore.ErrorLevel) - defer log.SetLevel(lvl) - - cols := []*types.FieldType{ - types.NewFieldType(mysql.TypeLonglong), - types.NewFieldType(mysql.TypeVarString), - } - - b.ReportAllocs() - cas := defaultHashJoinTestCase(cols, 0, false) - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkHashJoinExecWithCase(b, cas) - }) - - cas.keyIdx = []int{0} - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkHashJoinExecWithCase(b, cas) - }) - - cas.keyIdx = []int{0} - cas.disk = true - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkHashJoinExecWithCase(b, cas) - }) - - // Replace the wide string column with double column - cols = []*types.FieldType{ - types.NewFieldType(mysql.TypeLonglong), - types.NewFieldType(mysql.TypeDouble), - } - - cas = defaultHashJoinTestCase(cols, 0, false) - cas.keyIdx = []int{0} - cas.rows = 5 - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkHashJoinExecWithCase(b, cas) - }) - - cas = defaultHashJoinTestCase(cols, 0, false) - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkHashJoinExecWithCase(b, cas) - }) - - cas.keyIdx = []int{0} - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkHashJoinExecWithCase(b, cas) - }) -} - -func BenchmarkOuterHashJoinExec(b *testing.B) { - lvl := log.GetLevel() - log.SetLevel(zapcore.ErrorLevel) - defer log.SetLevel(lvl) - - cols := []*types.FieldType{ - types.NewFieldType(mysql.TypeLonglong), - types.NewFieldType(mysql.TypeVarString), - } - - b.ReportAllocs() - cas := defaultHashJoinTestCase(cols, 2, true) - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkHashJoinExecWithCase(b, cas) - }) - - cas.keyIdx = []int{0} - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkHashJoinExecWithCase(b, cas) - }) - - cas.keyIdx = []int{0} - cas.disk = true - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkHashJoinExecWithCase(b, cas) - }) - - // Replace the wide string column with double column - cols = []*types.FieldType{ - types.NewFieldType(mysql.TypeLonglong), - types.NewFieldType(mysql.TypeDouble), - } - - cas = defaultHashJoinTestCase(cols, 2, true) - cas.keyIdx = []int{0} - cas.rows = 5 - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkHashJoinExecWithCase(b, cas) - }) - - cas = defaultHashJoinTestCase(cols, 2, true) - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkHashJoinExecWithCase(b, cas) - }) - - cas.keyIdx = []int{0} - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkHashJoinExecWithCase(b, cas) - }) -} - -func benchmarkBuildHashTableForList(b *testing.B, casTest *hashJoinTestCase) { - opt := mockDataSourceParameters{ - schema: expression.NewSchema(casTest.columns()...), - rows: casTest.rows, - ctx: casTest.ctx, - genDataFunc: func(row int, typ *types.FieldType) interface{} { - switch typ.Tp { - case mysql.TypeLong, mysql.TypeLonglong: - return int64(row) - case mysql.TypeVarString: - return casTest.rawData - default: - panic("not implement") - } - }, - } - dataSource1 := buildMockDataSource(opt) - dataSource2 := buildMockDataSource(opt) - - dataSource1.prepareChunks() - b.ResetTimer() - for i := 0; i < b.N; i++ { - b.StopTimer() - exec := prepare4HashJoin(casTest, dataSource1, dataSource2) - tmpCtx := context.Background() - if err := exec.Open(tmpCtx); err != nil { - b.Fatal(err) - } - exec.prepared = true - - innerResultCh := make(chan *chunk.Chunk, len(dataSource1.chunks)) - for _, chk := range dataSource1.chunks { - innerResultCh <- chk - } - close(innerResultCh) - - b.StartTimer() - if err := exec.buildHashTableForList(innerResultCh); err != nil { - b.Fatal(err) - } - - if err := exec.Close(); err != nil { - b.Fatal(err) - } - b.StopTimer() - if exec.rowContainer.alreadySpilled() != casTest.disk { - b.Fatal("wrong usage with disk") - } - } -} - -func BenchmarkBuildHashTableForList(b *testing.B) { - lvl := log.GetLevel() - log.SetLevel(zapcore.ErrorLevel) - defer log.SetLevel(lvl) - - cols := []*types.FieldType{ - types.NewFieldType(mysql.TypeLonglong), - types.NewFieldType(mysql.TypeVarString), - } - - b.ReportAllocs() - cas := defaultHashJoinTestCase(cols, 0, false) - rows := []int{10, 100000} - keyIdxs := [][]int{{0, 1}, {0}} - disks := []bool{false, true} - for _, row := range rows { - for _, keyIdx := range keyIdxs { - for _, disk := range disks { - cas.rows = row - cas.keyIdx = keyIdx - cas.disk = disk - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkBuildHashTableForList(b, cas) - }) - } - } - } -} - -type indexJoinTestCase struct { - outerRows int - innerRows int - concurrency int - ctx sessionctx.Context - outerJoinKeyIdx []int - innerJoinKeyIdx []int - innerIdx []int - needOuterSort bool - rawData string -} - -func (tc indexJoinTestCase) columns() []*expression.Column { - return []*expression.Column{ - {Index: 0, RetType: types.NewFieldType(mysql.TypeLonglong)}, - {Index: 1, RetType: types.NewFieldType(mysql.TypeDouble)}, - {Index: 2, RetType: types.NewFieldType(mysql.TypeVarString)}, - } -} - -func defaultIndexJoinTestCase() *indexJoinTestCase { - ctx := mock.NewContext() - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize - ctx.GetSessionVars().SnapshotTS = 1 - ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, -1) - ctx.GetSessionVars().StmtCtx.DiskTracker = disk.NewTracker(nil, -1) - tc := &indexJoinTestCase{ - outerRows: 100000, - innerRows: variable.DefMaxChunkSize * 100, - concurrency: 4, - ctx: ctx, - outerJoinKeyIdx: []int{0, 1}, - innerJoinKeyIdx: []int{0, 1}, - innerIdx: []int{0, 1}, - rawData: wideString, - } - return tc -} - -func (tc indexJoinTestCase) String() string { - return fmt.Sprintf("(outerRows:%v, innerRows:%v, concurency:%v, outerJoinKeyIdx: %v, innerJoinKeyIdx: %v, NeedOuterSort:%v)", - tc.outerRows, tc.innerRows, tc.concurrency, tc.outerJoinKeyIdx, tc.innerJoinKeyIdx, tc.needOuterSort) -} -func (tc indexJoinTestCase) getMockDataSourceOptByRows(rows int) mockDataSourceParameters { - return mockDataSourceParameters{ - schema: expression.NewSchema(tc.columns()...), - rows: rows, - ctx: tc.ctx, - genDataFunc: func(row int, typ *types.FieldType) interface{} { - switch typ.Tp { - case mysql.TypeLong, mysql.TypeLonglong: - return int64(row) - case mysql.TypeDouble: - return float64(row) - case mysql.TypeVarString: - return tc.rawData - default: - panic("not implement") - } - }, - } -} - -func prepare4IndexInnerHashJoin(tc *indexJoinTestCase, outerDS *mockDataSource, innerDS *mockDataSource) Executor { - outerCols, innerCols := tc.columns(), tc.columns() - joinSchema := expression.NewSchema(outerCols...) - joinSchema.Append(innerCols...) - leftTypes, rightTypes := retTypes(outerDS), retTypes(innerDS) - defaultValues := make([]types.Datum, len(innerCols)) - colLens := make([]int, len(innerCols)) - for i := range colLens { - colLens[i] = types.UnspecifiedLength - } - keyOff2IdxOff := make([]int, len(tc.outerJoinKeyIdx)) - for i := range keyOff2IdxOff { - keyOff2IdxOff[i] = i - } - e := &IndexLookUpJoin{ - baseExecutor: newBaseExecutor(tc.ctx, joinSchema, stringutil.StringerStr("IndexInnerHashJoin"), outerDS), - outerCtx: outerCtx{ - rowTypes: leftTypes, - keyCols: tc.outerJoinKeyIdx, - }, - innerCtx: innerCtx{ - readerBuilder: &dataReaderBuilder{Plan: &mockPhysicalIndexReader{e: innerDS}, executorBuilder: newExecutorBuilder(tc.ctx, nil)}, - rowTypes: rightTypes, - colLens: colLens, - keyCols: tc.innerJoinKeyIdx, - }, - workerWg: new(sync.WaitGroup), - joiner: newJoiner(tc.ctx, 0, false, defaultValues, nil, leftTypes, rightTypes, nil), - isOuterJoin: false, - keyOff2IdxOff: keyOff2IdxOff, - lastColHelper: nil, - } - e.joinResult = newFirstChunk(e) - return e -} - -func prepare4IndexOuterHashJoin(tc *indexJoinTestCase, outerDS *mockDataSource, innerDS *mockDataSource) Executor { - e := prepare4IndexInnerHashJoin(tc, outerDS, innerDS).(*IndexLookUpJoin) - idxHash := &IndexNestedLoopHashJoin{IndexLookUpJoin: *e} - concurrency := tc.concurrency - idxHash.joiners = make([]joiner, concurrency) - for i := 0; i < concurrency; i++ { - idxHash.joiners[i] = e.joiner.Clone() - } - return idxHash -} - -func prepare4IndexMergeJoin(tc *indexJoinTestCase, outerDS *mockDataSource, innerDS *mockDataSource) Executor { - outerCols, innerCols := tc.columns(), tc.columns() - joinSchema := expression.NewSchema(outerCols...) - joinSchema.Append(innerCols...) - outerJoinKeys := make([]*expression.Column, 0, len(tc.outerJoinKeyIdx)) - innerJoinKeys := make([]*expression.Column, 0, len(tc.innerJoinKeyIdx)) - for _, keyIdx := range tc.outerJoinKeyIdx { - outerJoinKeys = append(outerJoinKeys, outerCols[keyIdx]) - } - for _, keyIdx := range tc.innerJoinKeyIdx { - innerJoinKeys = append(innerJoinKeys, innerCols[keyIdx]) - } - leftTypes, rightTypes := retTypes(outerDS), retTypes(innerDS) - defaultValues := make([]types.Datum, len(innerCols)) - colLens := make([]int, len(innerCols)) - for i := range colLens { - colLens[i] = types.UnspecifiedLength - } - keyOff2IdxOff := make([]int, len(outerJoinKeys)) - for i := range keyOff2IdxOff { - keyOff2IdxOff[i] = i - } - - compareFuncs := make([]expression.CompareFunc, 0, len(outerJoinKeys)) - outerCompareFuncs := make([]expression.CompareFunc, 0, len(outerJoinKeys)) - for i := range outerJoinKeys { - compareFuncs = append(compareFuncs, expression.GetCmpFunction(nil, outerJoinKeys[i], innerJoinKeys[i])) - outerCompareFuncs = append(outerCompareFuncs, expression.GetCmpFunction(nil, outerJoinKeys[i], outerJoinKeys[i])) - } - e := &IndexLookUpMergeJoin{ - baseExecutor: newBaseExecutor(tc.ctx, joinSchema, stringutil.StringerStr("IndexMergeJoin"), outerDS), - outerMergeCtx: outerMergeCtx{ - rowTypes: leftTypes, - keyCols: tc.outerJoinKeyIdx, - joinKeys: outerJoinKeys, - needOuterSort: tc.needOuterSort, - compareFuncs: outerCompareFuncs, - }, - innerMergeCtx: innerMergeCtx{ - readerBuilder: &dataReaderBuilder{Plan: &mockPhysicalIndexReader{e: innerDS}, executorBuilder: newExecutorBuilder(tc.ctx, nil)}, - rowTypes: rightTypes, - joinKeys: innerJoinKeys, - colLens: colLens, - keyCols: tc.innerJoinKeyIdx, - compareFuncs: compareFuncs, - }, - workerWg: new(sync.WaitGroup), - isOuterJoin: false, - keyOff2IdxOff: keyOff2IdxOff, - lastColHelper: nil, - } - joiners := make([]joiner, e.ctx.GetSessionVars().IndexLookupJoinConcurrency) - for i := 0; i < e.ctx.GetSessionVars().IndexLookupJoinConcurrency; i++ { - joiners[i] = newJoiner(tc.ctx, 0, false, defaultValues, nil, leftTypes, rightTypes, nil) - } - e.joiners = joiners - return e -} - -type indexJoinType int8 - -const ( - indexInnerHashJoin indexJoinType = iota - indexOuterHashJoin - indexMergeJoin -) - -func benchmarkIndexJoinExecWithCase( - b *testing.B, - tc *indexJoinTestCase, - outerDS *mockDataSource, - innerDS *mockDataSource, - execType indexJoinType, -) { - b.ResetTimer() - for i := 0; i < b.N; i++ { - b.StopTimer() - var exec Executor - switch execType { - case indexInnerHashJoin: - exec = prepare4IndexInnerHashJoin(tc, outerDS, innerDS) - case indexOuterHashJoin: - exec = prepare4IndexOuterHashJoin(tc, outerDS, innerDS) - case indexMergeJoin: - exec = prepare4IndexMergeJoin(tc, outerDS, innerDS) - } - - tmpCtx := context.Background() - chk := newFirstChunk(exec) - outerDS.prepareChunks() - innerDS.prepareChunks() - - b.StartTimer() - if err := exec.Open(tmpCtx); err != nil { - b.Fatal(err) - } - for { - if err := exec.Next(tmpCtx, chk); err != nil { - b.Fatal(err) - } - if chk.NumRows() == 0 { - break - } - } - - if err := exec.Close(); err != nil { - b.Fatal(err) - } - b.StopTimer() - } -} - -func BenchmarkIndexJoinExec(b *testing.B) { - lvl := log.GetLevel() - log.SetLevel(zapcore.ErrorLevel) - defer log.SetLevel(lvl) - - b.ReportAllocs() - tc := defaultIndexJoinTestCase() - outerOpt := tc.getMockDataSourceOptByRows(tc.outerRows) - innerOpt := tc.getMockDataSourceOptByRows(tc.innerRows) - outerDS := buildMockDataSourceWithIndex(outerOpt, tc.innerIdx) - innerDS := buildMockDataSourceWithIndex(innerOpt, tc.innerIdx) - - tc.needOuterSort = true - b.Run(fmt.Sprintf("index merge join need outer sort %v", tc), func(b *testing.B) { - benchmarkIndexJoinExecWithCase(b, tc, outerDS, innerDS, indexMergeJoin) - }) - - tc.needOuterSort = false - b.Run(fmt.Sprintf("index merge join %v", tc), func(b *testing.B) { - benchmarkIndexJoinExecWithCase(b, tc, outerDS, innerDS, indexMergeJoin) - }) - - b.Run(fmt.Sprintf("index inner hash join %v", tc), func(b *testing.B) { - benchmarkIndexJoinExecWithCase(b, tc, outerDS, innerDS, indexInnerHashJoin) - }) - - b.Run(fmt.Sprintf("index outer hash join %v", tc), func(b *testing.B) { - benchmarkIndexJoinExecWithCase(b, tc, outerDS, innerDS, indexOuterHashJoin) - }) -} - -type mergeJoinTestCase struct { - indexJoinTestCase - childrenUsedSchema [][]bool -} - -func prepare4MergeJoin(tc *mergeJoinTestCase, leftExec, rightExec *mockDataSource) *MergeJoinExec { - outerCols, innerCols := tc.columns(), tc.columns() - - joinSchema := expression.NewSchema() - if tc.childrenUsedSchema != nil { - for i, used := range tc.childrenUsedSchema[0] { - if used { - joinSchema.Append(outerCols[i]) - } - } - for i, used := range tc.childrenUsedSchema[1] { - if used { - joinSchema.Append(innerCols[i]) - } - } - } else { - joinSchema.Append(outerCols...) - joinSchema.Append(innerCols...) - } - - outerJoinKeys := make([]*expression.Column, 0, len(tc.outerJoinKeyIdx)) - innerJoinKeys := make([]*expression.Column, 0, len(tc.innerJoinKeyIdx)) - for _, keyIdx := range tc.outerJoinKeyIdx { - outerJoinKeys = append(outerJoinKeys, outerCols[keyIdx]) - } - for _, keyIdx := range tc.innerJoinKeyIdx { - innerJoinKeys = append(innerJoinKeys, innerCols[keyIdx]) - } - compareFuncs := make([]expression.CompareFunc, 0, len(outerJoinKeys)) - outerCompareFuncs := make([]expression.CompareFunc, 0, len(outerJoinKeys)) - for i := range outerJoinKeys { - compareFuncs = append(compareFuncs, expression.GetCmpFunction(nil, outerJoinKeys[i], innerJoinKeys[i])) - outerCompareFuncs = append(outerCompareFuncs, expression.GetCmpFunction(nil, outerJoinKeys[i], outerJoinKeys[i])) - } - - defaultValues := make([]types.Datum, len(innerCols)) - - // only benchmark inner join - e := &MergeJoinExec{ - stmtCtx: tc.ctx.GetSessionVars().StmtCtx, - baseExecutor: newBaseExecutor(tc.ctx, joinSchema, stringutil.StringerStr("MergeJoin"), leftExec, rightExec), - compareFuncs: compareFuncs, - isOuterJoin: false, - } - - e.joiner = newJoiner( - tc.ctx, - 0, - false, - defaultValues, - nil, - retTypes(leftExec), - retTypes(rightExec), - tc.childrenUsedSchema, - ) - - e.innerTable = &mergeJoinTable{ - isInner: true, - childIndex: 1, - joinKeys: innerJoinKeys, - } - - e.outerTable = &mergeJoinTable{ - childIndex: 0, - filters: nil, - joinKeys: outerJoinKeys, - } - - return e -} - -func defaultMergeJoinTestCase() *mergeJoinTestCase { - return &mergeJoinTestCase{*defaultIndexJoinTestCase(), nil} -} - -func newMergeJoinBenchmark(numOuterRows, numInnerDup, numInnerRedundant int) (tc *mergeJoinTestCase, innerDS, outerDS *mockDataSource) { - ctx := mock.NewContext() - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize - ctx.GetSessionVars().SnapshotTS = 1 - ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, -1) - ctx.GetSessionVars().StmtCtx.DiskTracker = disk.NewTracker(nil, -1) - - numInnerRows := numOuterRows*numInnerDup + numInnerRedundant - itc := &indexJoinTestCase{ - outerRows: numOuterRows, - innerRows: numInnerRows, - concurrency: 4, - ctx: ctx, - outerJoinKeyIdx: []int{0, 1}, - innerJoinKeyIdx: []int{0, 1}, - innerIdx: []int{0, 1}, - rawData: wideString, - } - tc = &mergeJoinTestCase{*itc, nil} - outerOpt := mockDataSourceParameters{ - schema: expression.NewSchema(tc.columns()...), - rows: numOuterRows, - ctx: tc.ctx, - genDataFunc: func(row int, typ *types.FieldType) interface{} { - switch typ.Tp { - case mysql.TypeLong, mysql.TypeLonglong: - return int64(row) - case mysql.TypeDouble: - return float64(row) - case mysql.TypeVarString: - return tc.rawData - default: - panic("not implement") - } - }, - } - - innerOpt := mockDataSourceParameters{ - schema: expression.NewSchema(tc.columns()...), - rows: numInnerRows, - ctx: tc.ctx, - genDataFunc: func(row int, typ *types.FieldType) interface{} { - row = row / numInnerDup - switch typ.Tp { - case mysql.TypeLong, mysql.TypeLonglong: - return int64(row) - case mysql.TypeDouble: - return float64(row) - case mysql.TypeVarString: - return tc.rawData - default: - panic("not implement") - } - }, - } - - innerDS = buildMockDataSource(innerOpt) - outerDS = buildMockDataSource(outerOpt) - - return -} - -type mergeJoinType int8 - -const ( - innerMergeJoin mergeJoinType = iota -) - -func benchmarkMergeJoinExecWithCase(b *testing.B, tc *mergeJoinTestCase, innerDS, outerDS *mockDataSource, joinType mergeJoinType) { - b.ResetTimer() - for i := 0; i < b.N; i++ { - b.StopTimer() - var exec Executor - switch joinType { - case innerMergeJoin: - exec = prepare4MergeJoin(tc, innerDS, outerDS) - } - - tmpCtx := context.Background() - chk := newFirstChunk(exec) - outerDS.prepareChunks() - innerDS.prepareChunks() - - b.StartTimer() - if err := exec.Open(tmpCtx); err != nil { - b.Fatal(err) - } - for { - if err := exec.Next(tmpCtx, chk); err != nil { - b.Fatal(err) - } - if chk.NumRows() == 0 { - break - } - } - - if err := exec.Close(); err != nil { - b.Fatal(err) - } - b.StopTimer() - } -} - -func BenchmarkMergeJoinExec(b *testing.B) { - lvl := log.GetLevel() - log.SetLevel(zapcore.ErrorLevel) - defer log.SetLevel(lvl) - b.ReportAllocs() - - totalRows := 300000 - - innerDupAndRedundant := [][]int{ - {1, 0}, - {100, 0}, - {10000, 0}, - {1, 30000}, - } - - childrenUsedSchemas := [][][]bool{ - nil, - { - {true, false, false}, - {false, true, false}, - }, - } - - for _, params := range innerDupAndRedundant { - numInnerDup, numInnerRedundant := params[0], params[1] - for _, childrenUsedSchema := range childrenUsedSchemas { - tc, innerDS, outerDS := newMergeJoinBenchmark(totalRows/numInnerDup, numInnerDup, numInnerRedundant) - inlineProj := false - if childrenUsedSchema != nil { - inlineProj = true - tc.childrenUsedSchema = childrenUsedSchema - } - - b.Run(fmt.Sprintf("merge join %v InlineProj:%v", tc, inlineProj), func(b *testing.B) { - benchmarkMergeJoinExecWithCase(b, tc, outerDS, innerDS, innerMergeJoin) - }) - } - } -} - -type sortCase struct { - rows int - orderByIdx []int - ndvs []int - ctx sessionctx.Context -} - -func (tc sortCase) columns() []*expression.Column { - return []*expression.Column{ - {Index: 0, RetType: types.NewFieldType(mysql.TypeLonglong)}, - {Index: 1, RetType: types.NewFieldType(mysql.TypeLonglong)}, - } -} - -func (tc sortCase) String() string { - return fmt.Sprintf("(rows:%v, orderBy:%v, ndvs: %v)", tc.rows, tc.orderByIdx, tc.ndvs) -} - -func defaultSortTestCase() *sortCase { - ctx := mock.NewContext() - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize - ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, -1) - tc := &sortCase{rows: 300000, orderByIdx: []int{0, 1}, ndvs: []int{0, 0}, ctx: ctx} - return tc -} - -func benchmarkSortExec(b *testing.B, cas *sortCase) { - opt := mockDataSourceParameters{ - schema: expression.NewSchema(cas.columns()...), - rows: cas.rows, - ctx: cas.ctx, - ndvs: cas.ndvs, - } - dataSource := buildMockDataSource(opt) - exec := &SortExec{ - baseExecutor: newBaseExecutor(cas.ctx, dataSource.schema, stringutil.StringerStr("sort"), dataSource), - ByItems: make([]*util.ByItems, 0, len(cas.orderByIdx)), - schema: dataSource.schema, - } - for _, idx := range cas.orderByIdx { - exec.ByItems = append(exec.ByItems, &util.ByItems{Expr: cas.columns()[idx]}) - } - b.ResetTimer() - for i := 0; i < b.N; i++ { - b.StopTimer() - tmpCtx := context.Background() - chk := newFirstChunk(exec) - dataSource.prepareChunks() - - b.StartTimer() - if err := exec.Open(tmpCtx); err != nil { - b.Fatal(err) - } - for { - if err := exec.Next(tmpCtx, chk); err != nil { - b.Fatal(err) - } - if chk.NumRows() == 0 { - break - } - } - - if err := exec.Close(); err != nil { - b.Fatal(err) - } - b.StopTimer() - } -} - -func BenchmarkSortExec(b *testing.B) { - b.ReportAllocs() - cas := defaultSortTestCase() - // all random data - cas.ndvs = []int{0, 0} - cas.orderByIdx = []int{0, 1} - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkSortExec(b, cas) - }) - - ndvs := []int{1, 10000} - for _, ndv := range ndvs { - cas.ndvs = []int{ndv, 0} - cas.orderByIdx = []int{0, 1} - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkSortExec(b, cas) - }) - - cas.ndvs = []int{ndv, 0} - cas.orderByIdx = []int{0} - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkSortExec(b, cas) - }) - - cas.ndvs = []int{ndv, 0} - cas.orderByIdx = []int{1} - b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { - benchmarkSortExec(b, cas) - }) - } -} ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) diff --git a/executor/builder.go b/executor/builder.go index 5723eb7ef3bf4..56da866a3057f 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -38,7 +38,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" - plannerutil "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" @@ -817,22 +816,6 @@ func (b *executorBuilder) buildTrace(v *plannercore.Trace) Executor { builder: b, format: v.Format, } -<<<<<<< HEAD -======= - if t.format == plannercore.TraceFormatLog { - return &SortExec{ - baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID(), t), - ByItems: []*plannerutil.ByItems{ - {Expr: &expression.Column{ - Index: 0, - RetType: types.NewFieldType(mysql.TypeTimestamp), - }}, - }, - schema: v.Schema(), - } - } - return t ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } // buildExplain builds a explain executor. `e.rows` collects final result to `ExplainExec`. diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index 5e51630b3a4bc..9b1096619a79a 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -23,11 +23,6 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" -<<<<<<< HEAD -======= - plannerutil "github.com/pingcap/tidb/planner/util" - "github.com/pingcap/tidb/sessionctx/variable" ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" @@ -244,92 +239,3 @@ func assertEqualStrings(c *C, got []field, expect []string) { c.Assert(string(got[i].str), Equals, expect[i]) } } -<<<<<<< HEAD -======= - -func (s *testExecSerialSuite) TestSortSpillDisk(c *C) { - originCfg := config.GetGlobalConfig() - newConf := *originCfg - newConf.OOMUseTmpStorage = true - newConf.MemQuotaQuery = 1 - config.StoreGlobalConfig(&newConf) - defer config.StoreGlobalConfig(originCfg) - - ctx := mock.NewContext() - ctx.GetSessionVars().InitChunkSize = variable.DefMaxChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize - ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, -1) - cas := &sortCase{rows: 2048, orderByIdx: []int{0, 1}, ndvs: []int{0, 0}, ctx: ctx} - opt := mockDataSourceParameters{ - schema: expression.NewSchema(cas.columns()...), - rows: cas.rows, - ctx: cas.ctx, - ndvs: cas.ndvs, - } - dataSource := buildMockDataSource(opt) - exec := &SortExec{ - baseExecutor: newBaseExecutor(cas.ctx, dataSource.schema, stringutil.StringerStr("sort"), dataSource), - ByItems: make([]*plannerutil.ByItems, 0, len(cas.orderByIdx)), - schema: dataSource.schema, - } - for _, idx := range cas.orderByIdx { - exec.ByItems = append(exec.ByItems, &plannerutil.ByItems{Expr: cas.columns()[idx]}) - } - tmpCtx := context.Background() - chk := newFirstChunk(exec) - dataSource.prepareChunks() - err := exec.Open(tmpCtx) - c.Assert(err, IsNil) - for { - err = exec.Next(tmpCtx, chk) - c.Assert(err, IsNil) - if chk.NumRows() == 0 { - break - } - } - // Test only 1 partition and all data in memory. - c.Assert(len(exec.partitionList), Equals, 1) - c.Assert(exec.partitionList[0].AlreadySpilled(), Equals, false) - c.Assert(exec.partitionList[0].NumRow(), Equals, 2048) - err = exec.Close() - c.Assert(err, IsNil) - - ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, 1) - dataSource.prepareChunks() - err = exec.Open(tmpCtx) - c.Assert(err, IsNil) - for { - err = exec.Next(tmpCtx, chk) - c.Assert(err, IsNil) - if chk.NumRows() == 0 { - break - } - } - // Test 2 partitions and all data in disk. - c.Assert(len(exec.partitionList), Equals, 2) - c.Assert(exec.partitionList[0].AlreadySpilled(), Equals, true) - c.Assert(exec.partitionList[1].AlreadySpilled(), Equals, true) - c.Assert(exec.partitionList[0].NumRow(), Equals, 1024) - c.Assert(exec.partitionList[1].NumRow(), Equals, 1024) - err = exec.Close() - c.Assert(err, IsNil) - - ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, 24000) - dataSource.prepareChunks() - err = exec.Open(tmpCtx) - c.Assert(err, IsNil) - for { - err = exec.Next(tmpCtx, chk) - c.Assert(err, IsNil) - if chk.NumRows() == 0 { - break - } - } - // Test only 1 partition but spill disk. - c.Assert(len(exec.partitionList), Equals, 1) - c.Assert(exec.partitionList[0].AlreadySpilled(), Equals, true) - c.Assert(exec.partitionList[0].NumRow(), Equals, 2048) - err = exec.Close() - c.Assert(err, IsNil) -} ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) diff --git a/expression/aggregation/aggregation.go b/expression/aggregation/aggregation.go index 676f93f810e5f..ba98bb37bbe0c 100644 --- a/expression/aggregation/aggregation.go +++ b/expression/aggregation/aggregation.go @@ -250,31 +250,3 @@ func IsAllFirstRow(aggFuncs []*AggFuncDesc) bool { } return true } -<<<<<<< HEAD -======= - -// CheckAggPushDown checks whether an agg function can be pushed to storage. -func CheckAggPushDown(aggFunc *AggFuncDesc, storeType kv.StoreType) bool { - if len(aggFunc.OrderByItems) > 0 { - return false - } - ret := true - switch storeType { - case kv.TiFlash: - ret = CheckAggPushFlash(aggFunc) - } - if ret { - ret = expression.IsPushDownEnabled(strings.ToLower(aggFunc.Name), storeType) - } - return ret -} - -// CheckAggPushFlash checks whether an agg function can be pushed to flash storage. -func CheckAggPushFlash(aggFunc *AggFuncDesc) bool { - switch aggFunc.Name { - case ast.AggFuncSum, ast.AggFuncCount, ast.AggFuncMin, ast.AggFuncMax, ast.AggFuncAvg, ast.AggFuncFirstRow: - return true - } - return false -} ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) diff --git a/planner/cascades/enforcer_rules.go b/planner/cascades/enforcer_rules.go index a526bad3fe577..c706f9942234d 100644 --- a/planner/cascades/enforcer_rules.go +++ b/planner/cascades/enforcer_rules.go @@ -56,16 +56,9 @@ func (e *OrderEnforcer) NewProperty(prop *property.PhysicalProperty) (newProp *p // OnEnforce adds sort operator to satisfy required order property. func (e *OrderEnforcer) OnEnforce(reqProp *property.PhysicalProperty, child memo.Implementation) (impl memo.Implementation) { -<<<<<<< HEAD sort := &plannercore.PhysicalSort{ - ByItems: make([]*plannercore.ByItems, 0, len(reqProp.Items)), - } -======= - childPlan := child.GetPlan() - sort := plannercore.PhysicalSort{ ByItems: make([]*util.ByItems, 0, len(reqProp.Items)), - }.Init(childPlan.SCtx(), childPlan.Stats(), childPlan.SelectBlockOffset(), &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64}) ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) + } for _, item := range reqProp.Items { item := &util.ByItems{ Expr: item.Col, diff --git a/planner/cascades/transformation_rules.go b/planner/cascades/transformation_rules.go index 1637637520139..4b394d4021793 100644 --- a/planner/cascades/transformation_rules.go +++ b/planner/cascades/transformation_rules.go @@ -16,14 +16,6 @@ package cascades import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/planner/memo" -<<<<<<< HEAD -======= - "github.com/pingcap/tidb/planner/util" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/ranger" - "github.com/pingcap/tidb/util/set" ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) ) // Transformation defines the interface for the transformation rules. @@ -47,2196 +39,5 @@ var transformationMap = map[memo.Operand][]Transformation{ operandProject: []Transformation{ nil, }, -<<<<<<< HEAD */ -======= -} - -// PostTransformationBatch does the transformation which is related to -// the constraints of the execution engine of TiDB. -// For example, TopN/Sort only support `order by` columns in TiDB layer, -// as for scalar functions, we need to inject a Projection for them -// below the TopN/Sort. -var PostTransformationBatch = TransformationRuleBatch{ - memo.OperandProjection: { - NewRuleEliminateProjection(), - NewRuleMergeAdjacentProjection(), - }, - memo.OperandTopN: { - NewRuleInjectProjectionBelowTopN(), - }, -} - -type baseRule struct { - pattern *memo.Pattern -} - -// Match implements Transformation Interface. -func (r *baseRule) Match(expr *memo.ExprIter) bool { - return true -} - -// GetPattern implements Transformation Interface. -func (r *baseRule) GetPattern() *memo.Pattern { - return r.pattern -} - -// PushSelDownTableScan pushes the selection down to TableScan. -type PushSelDownTableScan struct { - baseRule -} - -// NewRulePushSelDownTableScan creates a new Transformation PushSelDownTableScan. -// The pattern of this rule is: `Selection -> TableScan` -func NewRulePushSelDownTableScan() Transformation { - rule := &PushSelDownTableScan{} - ts := memo.NewPattern(memo.OperandTableScan, memo.EngineTiKVOrTiFlash) - p := memo.BuildPattern(memo.OperandSelection, memo.EngineTiKVOrTiFlash, ts) - rule.pattern = p - return rule -} - -// OnTransform implements Transformation interface. -// -// It transforms `sel -> ts` to one of the following new exprs: -// 1. `newSel -> newTS` -// 2. `newTS` -// -// Filters of the old `sel` operator are removed if they are used to calculate -// the key ranges of the `ts` operator. -func (r *PushSelDownTableScan) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) - ts := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalTableScan) - if ts.Handle == nil { - return nil, false, false, nil - } - accesses, remained := ranger.DetachCondsForColumn(ts.SCtx(), sel.Conditions, ts.Handle) - if accesses == nil { - return nil, false, false, nil - } - newTblScan := plannercore.LogicalTableScan{ - Source: ts.Source, - Handle: ts.Handle, - AccessConds: ts.AccessConds.Shallow(), - }.Init(ts.SCtx(), ts.SelectBlockOffset()) - newTblScan.AccessConds = append(newTblScan.AccessConds, accesses...) - tblScanExpr := memo.NewGroupExpr(newTblScan) - if len(remained) == 0 { - // `sel -> ts` is transformed to `newTS`. - return []*memo.GroupExpr{tblScanExpr}, true, false, nil - } - schema := old.GetExpr().Group.Prop.Schema - tblScanGroup := memo.NewGroupWithSchema(tblScanExpr, schema) - newSel := plannercore.LogicalSelection{Conditions: remained}.Init(sel.SCtx(), sel.SelectBlockOffset()) - selExpr := memo.NewGroupExpr(newSel) - selExpr.Children = append(selExpr.Children, tblScanGroup) - // `sel -> ts` is transformed to `newSel ->newTS`. - return []*memo.GroupExpr{selExpr}, true, false, nil -} - -// PushSelDownIndexScan pushes a Selection down to IndexScan. -type PushSelDownIndexScan struct { - baseRule -} - -// NewRulePushSelDownIndexScan creates a new Transformation PushSelDownIndexScan. -// The pattern of this rule is `Selection -> IndexScan`. -func NewRulePushSelDownIndexScan() Transformation { - rule := &PushSelDownIndexScan{} - rule.pattern = memo.BuildPattern( - memo.OperandSelection, - memo.EngineTiKVOnly, - memo.NewPattern(memo.OperandIndexScan, memo.EngineTiKVOnly), - ) - return rule -} - -// OnTransform implements Transformation interface. -// It will transform `Selection -> IndexScan` to: -// `IndexScan(with a new access range)` or -// `Selection -> IndexScan(with a new access range)` -// or just keep the two GroupExprs unchanged. -func (r *PushSelDownIndexScan) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) - is := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalIndexScan) - if len(is.IdxCols) == 0 { - return nil, false, false, nil - } - conditions := sel.Conditions - if is.AccessConds != nil { - // If we have already pushed some conditions down here, - // we merge old AccessConds with new conditions, - // to make sure this rule can be applied more than once. - conditions = make([]expression.Expression, len(sel.Conditions)+len(is.AccessConds)) - copy(conditions, sel.Conditions) - copy(conditions[len(sel.Conditions):], is.AccessConds) - } - res, err := ranger.DetachCondAndBuildRangeForIndex(is.SCtx(), conditions, is.IdxCols, is.IdxColLens) - if err != nil { - return nil, false, false, err - } - if len(res.AccessConds) == len(is.AccessConds) { - // There is no condition can be pushed down as range, - // or the pushed down conditions are the same with before. - sameConds := true - for i := range res.AccessConds { - if !res.AccessConds[i].Equal(is.SCtx(), is.AccessConds[i]) { - sameConds = false - break - } - } - if sameConds { - return nil, false, false, nil - } - } - // TODO: `res` still has some unused fields: EqOrInCount, IsDNFCond. - newIs := plannercore.LogicalIndexScan{ - Source: is.Source, - IsDoubleRead: is.IsDoubleRead, - EqCondCount: res.EqCondCount, - AccessConds: res.AccessConds, - Ranges: res.Ranges, - Index: is.Index, - Columns: is.Columns, - FullIdxCols: is.FullIdxCols, - FullIdxColLens: is.FullIdxColLens, - IdxCols: is.IdxCols, - IdxColLens: is.IdxColLens, - }.Init(is.SCtx(), is.SelectBlockOffset()) - isExpr := memo.NewGroupExpr(newIs) - - if len(res.RemainedConds) == 0 { - return []*memo.GroupExpr{isExpr}, true, false, nil - } - isGroup := memo.NewGroupWithSchema(isExpr, old.Children[0].GetExpr().Group.Prop.Schema) - newSel := plannercore.LogicalSelection{Conditions: res.RemainedConds}.Init(sel.SCtx(), sel.SelectBlockOffset()) - selExpr := memo.NewGroupExpr(newSel) - selExpr.SetChildren(isGroup) - return []*memo.GroupExpr{selExpr}, true, false, nil -} - -// PushSelDownTiKVSingleGather pushes the selection down to child of TiKVSingleGather. -type PushSelDownTiKVSingleGather struct { - baseRule -} - -// NewRulePushSelDownTiKVSingleGather creates a new Transformation PushSelDownTiKVSingleGather. -// The pattern of this rule is `Selection -> TiKVSingleGather -> Any`. -func NewRulePushSelDownTiKVSingleGather() Transformation { - any := memo.NewPattern(memo.OperandAny, memo.EngineTiKVOrTiFlash) - tg := memo.BuildPattern(memo.OperandTiKVSingleGather, memo.EngineTiDBOnly, any) - p := memo.BuildPattern(memo.OperandSelection, memo.EngineTiDBOnly, tg) - - rule := &PushSelDownTiKVSingleGather{} - rule.pattern = p - return rule -} - -// OnTransform implements Transformation interface. -// -// It transforms `oldSel -> oldTg -> any` to one of the following new exprs: -// 1. `newTg -> pushedSel -> any` -// 2. `remainedSel -> newTg -> pushedSel -> any` -func (r *PushSelDownTiKVSingleGather) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) - sg := old.Children[0].GetExpr().ExprNode.(*plannercore.TiKVSingleGather) - childGroup := old.Children[0].Children[0].Group - var pushed, remained []expression.Expression - sctx := sg.SCtx() - pushed, remained = expression.PushDownExprs(sctx.GetSessionVars().StmtCtx, sel.Conditions, sctx.GetClient(), kv.TiKV) - if len(pushed) == 0 { - return nil, false, false, nil - } - pushedSel := plannercore.LogicalSelection{Conditions: pushed}.Init(sctx, sel.SelectBlockOffset()) - pushedSelExpr := memo.NewGroupExpr(pushedSel) - pushedSelExpr.Children = append(pushedSelExpr.Children, childGroup) - pushedSelGroup := memo.NewGroupWithSchema(pushedSelExpr, childGroup.Prop.Schema).SetEngineType(childGroup.EngineType) - // The field content of TiKVSingleGather would not be modified currently, so we - // just reference the same tg instead of making a copy of it. - // - // TODO: if we save pushed filters later in TiKVSingleGather, in order to do partition - // pruning or skyline pruning, we need to make a copy of the TiKVSingleGather here. - tblGatherExpr := memo.NewGroupExpr(sg) - tblGatherExpr.Children = append(tblGatherExpr.Children, pushedSelGroup) - if len(remained) == 0 { - // `oldSel -> oldTg -> any` is transformed to `newTg -> pushedSel -> any`. - return []*memo.GroupExpr{tblGatherExpr}, true, false, nil - } - tblGatherGroup := memo.NewGroupWithSchema(tblGatherExpr, pushedSelGroup.Prop.Schema) - remainedSel := plannercore.LogicalSelection{Conditions: remained}.Init(sel.SCtx(), sel.SelectBlockOffset()) - remainedSelExpr := memo.NewGroupExpr(remainedSel) - remainedSelExpr.Children = append(remainedSelExpr.Children, tblGatherGroup) - // `oldSel -> oldTg -> any` is transformed to `remainedSel -> newTg -> pushedSel -> any`. - return []*memo.GroupExpr{remainedSelExpr}, true, false, nil -} - -// EnumeratePaths converts DataSource to table scan and index scans. -type EnumeratePaths struct { - baseRule -} - -// NewRuleEnumeratePaths creates a new Transformation EnumeratePaths. -// The pattern of this rule is: `DataSource`. -func NewRuleEnumeratePaths() Transformation { - rule := &EnumeratePaths{} - rule.pattern = memo.NewPattern(memo.OperandDataSource, memo.EngineTiDBOnly) - return rule -} - -// OnTransform implements Transformation interface. -func (r *EnumeratePaths) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - ds := old.GetExpr().ExprNode.(*plannercore.DataSource) - gathers := ds.Convert2Gathers() - for _, gather := range gathers { - expr := memo.Convert2GroupExpr(gather) - expr.Children[0].SetEngineType(memo.EngineTiKV) - newExprs = append(newExprs, expr) - } - return newExprs, true, false, nil -} - -// PushAggDownGather splits Aggregation to two stages, final and partial1, -// and pushed the partial Aggregation down to the child of TiKVSingleGather. -type PushAggDownGather struct { - baseRule -} - -// NewRulePushAggDownGather creates a new Transformation PushAggDownGather. -// The pattern of this rule is: `Aggregation -> TiKVSingleGather`. -func NewRulePushAggDownGather() Transformation { - rule := &PushAggDownGather{} - rule.pattern = memo.BuildPattern( - memo.OperandAggregation, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandTiKVSingleGather, memo.EngineTiDBOnly), - ) - return rule -} - -// Match implements Transformation interface. -func (r *PushAggDownGather) Match(expr *memo.ExprIter) bool { - if expr.GetExpr().HasAppliedRule(r) { - return false - } - agg := expr.GetExpr().ExprNode.(*plannercore.LogicalAggregation) - for _, aggFunc := range agg.AggFuncs { - if aggFunc.Mode != aggregation.CompleteMode { - return false - } - } - if agg.HasDistinct() { - // TODO: remove this logic after the cost estimation of distinct pushdown is implemented. - // If AllowDistinctAggPushDown is set to true, we should not consider RootTask. - if !agg.SCtx().GetSessionVars().AllowDistinctAggPushDown { - return false - } - } - childEngine := expr.Children[0].GetExpr().Children[0].EngineType - if childEngine != memo.EngineTiKV { - // TODO: Remove this check when we have implemented TiFlashAggregation. - return false - } - return plannercore.CheckAggCanPushCop(agg.SCtx(), agg.AggFuncs, agg.GroupByItems, kv.TiKV) -} - -// OnTransform implements Transformation interface. -// It will transform `Agg->Gather` to `Agg(Final) -> Gather -> Agg(Partial1)`. -func (r *PushAggDownGather) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - agg := old.GetExpr().ExprNode.(*plannercore.LogicalAggregation) - aggSchema := old.GetExpr().Group.Prop.Schema - gather := old.Children[0].GetExpr().ExprNode.(*plannercore.TiKVSingleGather) - childGroup := old.Children[0].GetExpr().Children[0] - // The old Aggregation should stay unchanged for other transformation. - // So we build a new LogicalAggregation for the partialAgg. - aggFuncs := make([]*aggregation.AggFuncDesc, len(agg.AggFuncs)) - for i := range agg.AggFuncs { - aggFuncs[i] = agg.AggFuncs[i].Clone() - } - gbyItems := make([]expression.Expression, len(agg.GroupByItems)) - copy(gbyItems, agg.GroupByItems) - - partialPref, finalPref, funcMap := plannercore.BuildFinalModeAggregation(agg.SCtx(), - &plannercore.AggInfo{ - AggFuncs: aggFuncs, - GroupByItems: gbyItems, - Schema: aggSchema, - }, true) - // Remove unnecessary FirstRow. - partialPref.AggFuncs = - plannercore.RemoveUnnecessaryFirstRow(agg.SCtx(), finalPref.AggFuncs, finalPref.GroupByItems, partialPref.AggFuncs, partialPref.GroupByItems, partialPref.Schema, funcMap) - - partialAgg := plannercore.LogicalAggregation{ - AggFuncs: partialPref.AggFuncs, - GroupByItems: partialPref.GroupByItems, - }.Init(agg.SCtx(), agg.SelectBlockOffset()) - partialAgg.CopyAggHints(agg) - - finalAgg := plannercore.LogicalAggregation{ - AggFuncs: finalPref.AggFuncs, - GroupByItems: finalPref.GroupByItems, - }.Init(agg.SCtx(), agg.SelectBlockOffset()) - finalAgg.CopyAggHints(agg) - - partialAggExpr := memo.NewGroupExpr(partialAgg) - partialAggExpr.SetChildren(childGroup) - partialAggGroup := memo.NewGroupWithSchema(partialAggExpr, partialPref.Schema).SetEngineType(childGroup.EngineType) - gatherExpr := memo.NewGroupExpr(gather) - gatherExpr.SetChildren(partialAggGroup) - gatherGroup := memo.NewGroupWithSchema(gatherExpr, partialPref.Schema) - finalAggExpr := memo.NewGroupExpr(finalAgg) - finalAggExpr.SetChildren(gatherGroup) - finalAggExpr.AddAppliedRule(r) - // We don't erase the old complete mode Aggregation because - // this transformation would not always be better. - return []*memo.GroupExpr{finalAggExpr}, false, false, nil -} - -// PushSelDownSort pushes the Selection down to the child of Sort. -type PushSelDownSort struct { - baseRule -} - -// NewRulePushSelDownSort creates a new Transformation PushSelDownSort. -// The pattern of this rule is: `Selection -> Sort`. -func NewRulePushSelDownSort() Transformation { - rule := &PushSelDownSort{} - rule.pattern = memo.BuildPattern( - memo.OperandSelection, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandSort, memo.EngineTiDBOnly), - ) - return rule -} - -// OnTransform implements Transformation interface. -// It will transform `sel->sort->x` to `sort->sel->x`. -func (r *PushSelDownSort) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) - sort := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalSort) - childGroup := old.Children[0].GetExpr().Children[0] - - newSelExpr := memo.NewGroupExpr(sel) - newSelExpr.Children = append(newSelExpr.Children, childGroup) - newSelGroup := memo.NewGroupWithSchema(newSelExpr, childGroup.Prop.Schema) - - newSortExpr := memo.NewGroupExpr(sort) - newSortExpr.Children = append(newSortExpr.Children, newSelGroup) - return []*memo.GroupExpr{newSortExpr}, true, false, nil -} - -// PushSelDownProjection pushes the Selection down to the child of Projection. -type PushSelDownProjection struct { - baseRule -} - -// NewRulePushSelDownProjection creates a new Transformation PushSelDownProjection. -// The pattern of this rule is: `Selection -> Projection`. -func NewRulePushSelDownProjection() Transformation { - rule := &PushSelDownProjection{} - rule.pattern = memo.BuildPattern( - memo.OperandSelection, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandProjection, memo.EngineTiDBOnly), - ) - return rule -} - -// OnTransform implements Transformation interface. -// It will transform `selection -> projection -> x` to -// 1. `projection -> selection -> x` or -// 2. `selection -> projection -> selection -> x` or -// 3. just keep unchanged. -func (r *PushSelDownProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) - proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) - projSchema := old.Children[0].Prop.Schema - childGroup := old.Children[0].GetExpr().Children[0] - for _, expr := range proj.Exprs { - if expression.HasAssignSetVarFunc(expr) { - return nil, false, false, nil - } - } - canBePushed := make([]expression.Expression, 0, len(sel.Conditions)) - canNotBePushed := make([]expression.Expression, 0, len(sel.Conditions)) - for _, cond := range sel.Conditions { - if !expression.HasGetSetVarFunc(cond) { - canBePushed = append(canBePushed, expression.ColumnSubstitute(cond, projSchema, proj.Exprs)) - } else { - canNotBePushed = append(canNotBePushed, cond) - } - } - if len(canBePushed) == 0 { - return nil, false, false, nil - } - newBottomSel := plannercore.LogicalSelection{Conditions: canBePushed}.Init(sel.SCtx(), sel.SelectBlockOffset()) - newBottomSelExpr := memo.NewGroupExpr(newBottomSel) - newBottomSelExpr.SetChildren(childGroup) - newBottomSelGroup := memo.NewGroupWithSchema(newBottomSelExpr, childGroup.Prop.Schema) - newProjExpr := memo.NewGroupExpr(proj) - newProjExpr.SetChildren(newBottomSelGroup) - if len(canNotBePushed) == 0 { - return []*memo.GroupExpr{newProjExpr}, true, false, nil - } - newProjGroup := memo.NewGroupWithSchema(newProjExpr, projSchema) - newTopSel := plannercore.LogicalSelection{Conditions: canNotBePushed}.Init(sel.SCtx(), sel.SelectBlockOffset()) - newTopSelExpr := memo.NewGroupExpr(newTopSel) - newTopSelExpr.SetChildren(newProjGroup) - return []*memo.GroupExpr{newTopSelExpr}, true, false, nil -} - -// PushSelDownAggregation pushes Selection down to the child of Aggregation. -type PushSelDownAggregation struct { - baseRule -} - -// NewRulePushSelDownAggregation creates a new Transformation PushSelDownAggregation. -// The pattern of this rule is `Selection -> Aggregation`. -func NewRulePushSelDownAggregation() Transformation { - rule := &PushSelDownAggregation{} - rule.pattern = memo.BuildPattern( - memo.OperandSelection, - memo.EngineAll, - memo.NewPattern(memo.OperandAggregation, memo.EngineAll), - ) - return rule -} - -// OnTransform implements Transformation interface. -// It will transform `sel->agg->x` to `agg->sel->x` or `sel->agg->sel->x` -// or just keep the selection unchanged. -func (r *PushSelDownAggregation) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) - agg := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalAggregation) - aggSchema := old.Children[0].Prop.Schema - var pushedExprs []expression.Expression - var remainedExprs []expression.Expression - exprsOriginal := make([]expression.Expression, 0, len(agg.AggFuncs)) - for _, aggFunc := range agg.AggFuncs { - exprsOriginal = append(exprsOriginal, aggFunc.Args[0]) - } - groupByColumns := expression.NewSchema(agg.GetGroupByCols()...) - for _, cond := range sel.Conditions { - switch cond.(type) { - case *expression.Constant: - // Consider SQL list "select sum(b) from t group by a having 1=0". "1=0" is a constant predicate which should be - // retained and pushed down at the same time. Because we will get a wrong query result that contains one column - // with value 0 rather than an empty query result. - pushedExprs = append(pushedExprs, cond) - remainedExprs = append(remainedExprs, cond) - case *expression.ScalarFunction: - extractedCols := expression.ExtractColumns(cond) - canPush := true - for _, col := range extractedCols { - if !groupByColumns.Contains(col) { - canPush = false - break - } - } - if canPush { - pushedExprs = append(pushedExprs, cond) - } else { - remainedExprs = append(remainedExprs, cond) - } - default: - remainedExprs = append(remainedExprs, cond) - } - } - // If no condition can be pushed, keep the selection unchanged. - if len(pushedExprs) == 0 { - return nil, false, false, nil - } - sctx := sel.SCtx() - childGroup := old.Children[0].GetExpr().Children[0] - pushedSel := plannercore.LogicalSelection{Conditions: pushedExprs}.Init(sctx, sel.SelectBlockOffset()) - pushedGroupExpr := memo.NewGroupExpr(pushedSel) - pushedGroupExpr.SetChildren(childGroup) - pushedGroup := memo.NewGroupWithSchema(pushedGroupExpr, childGroup.Prop.Schema) - - aggGroupExpr := memo.NewGroupExpr(agg) - aggGroupExpr.SetChildren(pushedGroup) - - if len(remainedExprs) == 0 { - return []*memo.GroupExpr{aggGroupExpr}, true, false, nil - } - - aggGroup := memo.NewGroupWithSchema(aggGroupExpr, aggSchema) - remainedSel := plannercore.LogicalSelection{Conditions: remainedExprs}.Init(sctx, sel.SelectBlockOffset()) - remainedGroupExpr := memo.NewGroupExpr(remainedSel) - remainedGroupExpr.SetChildren(aggGroup) - return []*memo.GroupExpr{remainedGroupExpr}, true, false, nil -} - -// PushSelDownWindow pushes Selection down to the child of Window. -type PushSelDownWindow struct { - baseRule -} - -// NewRulePushSelDownWindow creates a new Transformation PushSelDownWindow. -// The pattern of this rule is `Selection -> Window`. -func NewRulePushSelDownWindow() Transformation { - rule := &PushSelDownWindow{} - rule.pattern = memo.BuildPattern( - memo.OperandSelection, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandWindow, memo.EngineAll), - ) - return rule -} - -// OnTransform implements Transformation interface. -// This rule will transform `sel -> window -> x` to -// 1. `window -> sel -> x` or -// 2. `sel -> window -> sel -> x` or -// 3. just keep unchanged. -func (r *PushSelDownWindow) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) - window := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalWindow) - windowSchema := old.Children[0].Prop.Schema - childGroup := old.Children[0].GetExpr().Children[0] - canBePushed := make([]expression.Expression, 0, len(sel.Conditions)) - canNotBePushed := make([]expression.Expression, 0, len(sel.Conditions)) - - // get partition Columns' Schema - partitionColsSchema := expression.NewSchema(window.GetPartitionByCols()...) - - for _, cond := range sel.Conditions { - if expression.ExprFromSchema(cond, partitionColsSchema) { - canBePushed = append(canBePushed, cond) - } else { - canNotBePushed = append(canNotBePushed, cond) - } - } - // Nothing can be pushed! - if len(canBePushed) == 0 { - return nil, false, false, nil - } - - // construct return GroupExpr - newBottomSel := plannercore.LogicalSelection{Conditions: canBePushed}.Init(sel.SCtx(), sel.SelectBlockOffset()) - newBottomSelExpr := memo.NewGroupExpr(newBottomSel) - newBottomSelExpr.SetChildren(childGroup) - newBottomSelGroup := memo.NewGroupWithSchema(newBottomSelExpr, childGroup.Prop.Schema) - newWindowExpr := memo.NewGroupExpr(window) - newWindowExpr.SetChildren(newBottomSelGroup) - if len(canNotBePushed) == 0 { - return []*memo.GroupExpr{newWindowExpr}, true, false, nil - } - - newWindowGroup := memo.NewGroupWithSchema(newWindowExpr, windowSchema) - newTopSel := plannercore.LogicalSelection{Conditions: canNotBePushed}.Init(sel.SCtx(), sel.SelectBlockOffset()) - newTopSelExpr := memo.NewGroupExpr(newTopSel) - newTopSelExpr.SetChildren(newWindowGroup) - return []*memo.GroupExpr{newTopSelExpr}, true, false, nil -} - -// TransformLimitToTopN transforms Limit+Sort to TopN. -type TransformLimitToTopN struct { - baseRule -} - -// NewRuleTransformLimitToTopN creates a new Transformation TransformLimitToTopN. -// The pattern of this rule is `Limit -> Sort`. -func NewRuleTransformLimitToTopN() Transformation { - rule := &TransformLimitToTopN{} - rule.pattern = memo.BuildPattern( - memo.OperandLimit, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandSort, memo.EngineTiDBOnly), - ) - return rule -} - -// 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 -} - -// PushLimitDownProjection pushes Limit to Projection. -type PushLimitDownProjection struct { - baseRule -} - -// NewRulePushLimitDownProjection creates a new Transformation. -// The pattern of this rule is `Limit->Projection->X` to `Projection->Limit->X`. -func NewRulePushLimitDownProjection() Transformation { - rule := &PushLimitDownProjection{} - rule.pattern = memo.BuildPattern( - memo.OperandLimit, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandProjection, memo.EngineTiDBOnly), - ) - return rule -} - -// Match implements Transformation interface. -func (r *PushLimitDownProjection) Match(expr *memo.ExprIter) bool { - proj := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) - for _, expr := range proj.Exprs { - if expression.HasAssignSetVarFunc(expr) { - return false - } - } - return true -} - -// OnTransform implements Transformation interface. -// This rule tries to pushes the Limit through Projection. -func (r *PushLimitDownProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - limit := old.GetExpr().ExprNode.(*plannercore.LogicalLimit) - proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) - childGroup := old.Children[0].GetExpr().Children[0] - - projExpr := memo.NewGroupExpr(proj) - limitExpr := memo.NewGroupExpr(limit) - limitExpr.SetChildren(childGroup) - limitGroup := memo.NewGroupWithSchema(limitExpr, childGroup.Prop.Schema) - projExpr.SetChildren(limitGroup) - return []*memo.GroupExpr{projExpr}, true, false, nil -} - -// PushLimitDownUnionAll pushes limit to union all. -type PushLimitDownUnionAll struct { - baseRule -} - -// NewRulePushLimitDownUnionAll creates a new Transformation PushLimitDownUnionAll. -// The pattern of this rule is `Limit->UnionAll->X`. -func NewRulePushLimitDownUnionAll() Transformation { - rule := &PushLimitDownUnionAll{} - rule.pattern = memo.BuildPattern( - memo.OperandLimit, - 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 *PushLimitDownUnionAll) Match(expr *memo.ExprIter) bool { - return !expr.GetExpr().HasAppliedRule(r) -} - -// OnTransform implements Transformation interface. -// It will transform `Limit->UnionAll->X` to `Limit->UnionAll->Limit->X`. -func (r *PushLimitDownUnionAll) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - limit := old.GetExpr().ExprNode.(*plannercore.LogicalLimit) - unionAll := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalUnionAll) - unionAllSchema := old.Children[0].Group.Prop.Schema - - newLimit := plannercore.LogicalLimit{ - Count: limit.Count + limit.Offset, - }.Init(limit.SCtx(), limit.SelectBlockOffset()) - - newUnionAllExpr := memo.NewGroupExpr(unionAll) - for _, childGroup := range old.Children[0].GetExpr().Children { - newLimitExpr := memo.NewGroupExpr(newLimit) - newLimitExpr.Children = append(newLimitExpr.Children, childGroup) - newLimitGroup := memo.NewGroupWithSchema(newLimitExpr, childGroup.Prop.Schema) - - newUnionAllExpr.Children = append(newUnionAllExpr.Children, newLimitGroup) - } - - newLimitExpr := memo.NewGroupExpr(limit) - newUnionAllGroup := memo.NewGroupWithSchema(newUnionAllExpr, unionAllSchema) - newLimitExpr.SetChildren(newUnionAllGroup) - newLimitExpr.AddAppliedRule(r) - return []*memo.GroupExpr{newLimitExpr}, true, false, nil -} - -// PushSelDownJoin pushes Selection through Join. -type PushSelDownJoin struct { - baseRule -} - -// NewRulePushSelDownJoin creates a new Transformation PushSelDownJoin. -// The pattern of this rule is `Selection -> Join`. -func NewRulePushSelDownJoin() Transformation { - rule := &PushSelDownJoin{} - rule.pattern = memo.BuildPattern( - memo.OperandSelection, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandJoin, memo.EngineTiDBOnly), - ) - return rule -} - -// Match implements Transformation interface. -func (r *PushSelDownJoin) Match(expr *memo.ExprIter) bool { - return !expr.GetExpr().HasAppliedRule(r) -} - -// buildChildSelectionGroup builds a new childGroup if the pushed down condition is not empty. -func buildChildSelectionGroup( - oldSel *plannercore.LogicalSelection, - conditions []expression.Expression, - childGroup *memo.Group) *memo.Group { - if len(conditions) == 0 { - return childGroup - } - newSel := plannercore.LogicalSelection{Conditions: conditions}.Init(oldSel.SCtx(), oldSel.SelectBlockOffset()) - groupExpr := memo.NewGroupExpr(newSel) - groupExpr.SetChildren(childGroup) - newChild := memo.NewGroupWithSchema(groupExpr, childGroup.Prop.Schema) - return newChild -} - -// OnTransform implements Transformation interface. -// This rule tries to pushes the Selection through Join. Besides, this rule fulfills the `XXXConditions` field of Join. -func (r *PushSelDownJoin) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) - joinExpr := old.Children[0].GetExpr() - // TODO: we need to create a new LogicalJoin here. - join := joinExpr.ExprNode.(*plannercore.LogicalJoin) - sctx := sel.SCtx() - leftGroup := old.Children[0].GetExpr().Children[0] - rightGroup := old.Children[0].GetExpr().Children[1] - var equalCond []*expression.ScalarFunction - var leftPushCond, rightPushCond, otherCond, leftCond, rightCond, remainCond []expression.Expression - switch join.JoinType { - case plannercore.SemiJoin, plannercore.InnerJoin: - tempCond := make([]expression.Expression, 0, - len(join.LeftConditions)+len(join.RightConditions)+len(join.EqualConditions)+len(join.OtherConditions)+len(sel.Conditions)) - tempCond = append(tempCond, join.LeftConditions...) - tempCond = append(tempCond, join.RightConditions...) - tempCond = append(tempCond, expression.ScalarFuncs2Exprs(join.EqualConditions)...) - tempCond = append(tempCond, join.OtherConditions...) - tempCond = append(tempCond, sel.Conditions...) - tempCond = expression.ExtractFiltersFromDNFs(sctx, tempCond) - tempCond = expression.PropagateConstant(sctx, tempCond) - // Return table dual when filter is constant false or null. - dual := plannercore.Conds2TableDual(join, tempCond) - if dual != nil { - return []*memo.GroupExpr{memo.NewGroupExpr(dual)}, false, true, nil - } - equalCond, leftPushCond, rightPushCond, otherCond = join.ExtractOnCondition(tempCond, leftGroup.Prop.Schema, rightGroup.Prop.Schema, true, true) - join.LeftConditions = nil - join.RightConditions = nil - join.EqualConditions = equalCond - join.OtherConditions = otherCond - leftCond = leftPushCond - rightCond = rightPushCond - case plannercore.LeftOuterJoin, plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin, - plannercore.RightOuterJoin: - lenJoinConds := len(join.EqualConditions) + len(join.LeftConditions) + len(join.RightConditions) + len(join.OtherConditions) - joinConds := make([]expression.Expression, 0, lenJoinConds) - for _, equalCond := range join.EqualConditions { - joinConds = append(joinConds, equalCond) - } - joinConds = append(joinConds, join.LeftConditions...) - joinConds = append(joinConds, join.RightConditions...) - joinConds = append(joinConds, join.OtherConditions...) - join.EqualConditions = nil - join.LeftConditions = nil - join.RightConditions = nil - join.OtherConditions = nil - remainCond = make([]expression.Expression, len(sel.Conditions)) - copy(remainCond, sel.Conditions) - nullSensitive := join.JoinType == plannercore.AntiLeftOuterSemiJoin || join.JoinType == plannercore.LeftOuterSemiJoin - if join.JoinType == plannercore.RightOuterJoin { - joinConds, remainCond = expression.PropConstOverOuterJoin(join.SCtx(), joinConds, remainCond, rightGroup.Prop.Schema, leftGroup.Prop.Schema, nullSensitive) - } else { - joinConds, remainCond = expression.PropConstOverOuterJoin(join.SCtx(), joinConds, remainCond, leftGroup.Prop.Schema, rightGroup.Prop.Schema, nullSensitive) - } - eq, left, right, other := join.ExtractOnCondition(joinConds, leftGroup.Prop.Schema, rightGroup.Prop.Schema, false, false) - join.AppendJoinConds(eq, left, right, other) - // Return table dual when filter is constant false or null. - dual := plannercore.Conds2TableDual(join, remainCond) - if dual != nil { - return []*memo.GroupExpr{memo.NewGroupExpr(dual)}, false, true, nil - } - if join.JoinType == plannercore.RightOuterJoin { - remainCond = expression.ExtractFiltersFromDNFs(join.SCtx(), remainCond) - // Only derive right where condition, because left where condition cannot be pushed down - equalCond, leftPushCond, rightPushCond, otherCond = join.ExtractOnCondition(remainCond, leftGroup.Prop.Schema, rightGroup.Prop.Schema, false, true) - rightCond = rightPushCond - // Handle join conditions, only derive left join condition, because right join condition cannot be pushed down - derivedLeftJoinCond, _ := plannercore.DeriveOtherConditions(join, true, false) - leftCond = append(join.LeftConditions, derivedLeftJoinCond...) - join.LeftConditions = nil - remainCond = append(expression.ScalarFuncs2Exprs(equalCond), otherCond...) - remainCond = append(remainCond, leftPushCond...) - } else { - remainCond = expression.ExtractFiltersFromDNFs(join.SCtx(), remainCond) - // Only derive left where condition, because right where condition cannot be pushed down - equalCond, leftPushCond, rightPushCond, otherCond = join.ExtractOnCondition(remainCond, leftGroup.Prop.Schema, rightGroup.Prop.Schema, true, false) - leftCond = leftPushCond - // Handle join conditions, only derive left join condition, because right join condition cannot be pushed down - _, derivedRightJoinCond := plannercore.DeriveOtherConditions(join, false, true) - rightCond = append(join.RightConditions, derivedRightJoinCond...) - join.RightConditions = nil - remainCond = append(expression.ScalarFuncs2Exprs(equalCond), otherCond...) - remainCond = append(remainCond, rightPushCond...) - } - default: - // TODO: Enhance this rule to deal with Semi/SmiAnti Joins. - } - leftCond = expression.RemoveDupExprs(sctx, leftCond) - rightCond = expression.RemoveDupExprs(sctx, rightCond) - // TODO: Update EqualConditions like what we have done in the method join.updateEQCond() before. - leftGroup = buildChildSelectionGroup(sel, leftCond, leftGroup) - rightGroup = buildChildSelectionGroup(sel, rightCond, rightGroup) - newJoinExpr := memo.NewGroupExpr(join) - newJoinExpr.SetChildren(leftGroup, rightGroup) - if len(remainCond) > 0 { - newSel := plannercore.LogicalSelection{Conditions: remainCond}.Init(sctx, sel.SelectBlockOffset()) - newSel.Conditions = remainCond - newSelExpr := memo.NewGroupExpr(newSel) - newSelExpr.SetChildren(memo.NewGroupWithSchema(newJoinExpr, old.Children[0].Prop.Schema)) - newSelExpr.AddAppliedRule(r) - return []*memo.GroupExpr{newSelExpr}, true, false, nil - } - return []*memo.GroupExpr{newJoinExpr}, true, false, nil -} - -// PushSelDownUnionAll pushes selection through union all. -type PushSelDownUnionAll struct { - baseRule -} - -// NewRulePushSelDownUnionAll creates a new Transformation PushSelDownUnionAll. -// The pattern of this rule is `Selection -> UnionAll`. -func NewRulePushSelDownUnionAll() Transformation { - rule := &PushSelDownUnionAll{} - rule.pattern = memo.BuildPattern( - memo.OperandSelection, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandUnionAll, memo.EngineTiDBOnly), - ) - return rule -} - -// OnTransform implements Transformation interface. -// It will transform `Selection->UnionAll->x` to `UnionAll->Selection->x`. -func (r *PushSelDownUnionAll) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) - unionAll := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalUnionAll) - childGroups := old.Children[0].GetExpr().Children - - newUnionAllExpr := memo.NewGroupExpr(unionAll) - for _, group := range childGroups { - newSelExpr := memo.NewGroupExpr(sel) - newSelExpr.Children = append(newSelExpr.Children, group) - newSelGroup := memo.NewGroupWithSchema(newSelExpr, group.Prop.Schema) - - newUnionAllExpr.Children = append(newUnionAllExpr.Children, newSelGroup) - } - return []*memo.GroupExpr{newUnionAllExpr}, true, false, nil -} - -// EliminateProjection eliminates the projection. -type EliminateProjection struct { - baseRule -} - -// NewRuleEliminateProjection creates a new Transformation EliminateProjection. -// The pattern of this rule is `Projection -> Any`. -func NewRuleEliminateProjection() Transformation { - rule := &EliminateProjection{} - rule.pattern = memo.BuildPattern( - memo.OperandProjection, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandAny, memo.EngineTiDBOnly), - ) - return rule -} - -// OnTransform implements Transformation interface. -// This rule tries to eliminate the projection whose output columns are the same with its child. -func (r *EliminateProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - child := old.Children[0] - if child.Group.Prop.Schema.Len() != old.GetExpr().Group.Prop.Schema.Len() { - return nil, false, false, nil - } - - oldCols := old.GetExpr().Group.Prop.Schema.Columns - for i, col := range child.Group.Prop.Schema.Columns { - if !col.Equal(nil, oldCols[i]) { - return nil, false, false, nil - } - } - - // Promote the children group's expression. - finalGroupExprs := make([]*memo.GroupExpr, 0, child.Group.Equivalents.Len()) - for elem := child.Group.Equivalents.Front(); elem != nil; elem = elem.Next() { - childExpr := elem.Value.(*memo.GroupExpr) - copyChildExpr := memo.NewGroupExpr(childExpr.ExprNode) - copyChildExpr.SetChildren(childExpr.Children...) - finalGroupExprs = append(finalGroupExprs, copyChildExpr) - } - return finalGroupExprs, true, false, nil -} - -// MergeAdjacentProjection merge the adjacent projection. -type MergeAdjacentProjection struct { - baseRule -} - -// NewRuleMergeAdjacentProjection creates a new Transformation MergeAdjacentProjection. -// The pattern of this rule is `Projection -> Projection`. -func NewRuleMergeAdjacentProjection() Transformation { - rule := &MergeAdjacentProjection{} - rule.pattern = memo.BuildPattern( - memo.OperandProjection, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandProjection, memo.EngineTiDBOnly), - ) - return rule -} - -// OnTransform implements Transformation interface. -// It will transform `proj->proj->x` to `proj->x` -// or just keep the adjacent projections unchanged. -func (r *MergeAdjacentProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - proj := old.GetExpr().ExprNode.(*plannercore.LogicalProjection) - childGroup := old.Children[0].Group - child := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) - if plannercore.ExprsHasSideEffects(child.Exprs) { - return nil, false, false, nil - } - - replace := make(map[string]*expression.Column) - for i, col := range childGroup.Prop.Schema.Columns { - if colOrigin, ok := child.Exprs[i].(*expression.Column); ok { - replace[string(col.HashCode(nil))] = colOrigin - } - } - - newProj := plannercore.LogicalProjection{Exprs: make([]expression.Expression, len(proj.Exprs))}.Init(proj.SCtx(), proj.SelectBlockOffset()) - newProj.SetSchema(old.GetExpr().Group.Prop.Schema) - for i, expr := range proj.Exprs { - newExpr := expr.Clone() - plannercore.ResolveExprAndReplace(newExpr, replace) - newProj.Exprs[i] = plannercore.ReplaceColumnOfExpr(newExpr, child, childGroup.Prop.Schema) - } - - newProjExpr := memo.NewGroupExpr(newProj) - newProjExpr.SetChildren(old.Children[0].GetExpr().Children[0]) - return []*memo.GroupExpr{newProjExpr}, true, false, nil -} - -// PushTopNDownOuterJoin pushes topN to outer join. -type PushTopNDownOuterJoin struct { - baseRule -} - -// NewRulePushTopNDownOuterJoin creates a new Transformation PushTopNDownOuterJoin. -// The pattern of this rule is: `TopN -> Join`. -func NewRulePushTopNDownOuterJoin() Transformation { - rule := &PushTopNDownOuterJoin{} - rule.pattern = memo.BuildPattern( - memo.OperandTopN, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandJoin, memo.EngineTiDBOnly), - ) - return rule -} - -// Match implements Transformation interface. -// Use appliedRuleSet in GroupExpr to avoid re-apply rules. -func (r *PushTopNDownOuterJoin) Match(expr *memo.ExprIter) bool { - if expr.GetExpr().HasAppliedRule(r) { - return false - } - join := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin) - switch join.JoinType { - case plannercore.LeftOuterJoin, plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin, plannercore.RightOuterJoin: - return true - default: - return false - } -} - -func pushTopNDownOuterJoinToChild(topN *plannercore.LogicalTopN, outerGroup *memo.Group) *memo.Group { - for _, by := range topN.ByItems { - cols := expression.ExtractColumns(by.Expr) - for _, col := range cols { - if !outerGroup.Prop.Schema.Contains(col) { - return outerGroup - } - } - } - - newTopN := plannercore.LogicalTopN{ - Count: topN.Count + topN.Offset, - ByItems: make([]*util.ByItems, len(topN.ByItems)), - }.Init(topN.SCtx(), topN.SelectBlockOffset()) - - for i := range topN.ByItems { - newTopN.ByItems[i] = topN.ByItems[i].Clone() - } - newTopNGroup := memo.NewGroupExpr(newTopN) - newTopNGroup.SetChildren(outerGroup) - newChild := memo.NewGroupWithSchema(newTopNGroup, outerGroup.Prop.Schema) - return newChild -} - -// OnTransform implements Transformation interface. -// This rule will transform `TopN->OuterJoin->(OuterChild, InnerChild)` to `TopN->OuterJoin->(TopN->OuterChild, InnerChild)` -func (r *PushTopNDownOuterJoin) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - topN := old.GetExpr().ExprNode.(*plannercore.LogicalTopN) - joinExpr := old.Children[0].GetExpr() - join := joinExpr.ExprNode.(*plannercore.LogicalJoin) - joinSchema := old.Children[0].Group.Prop.Schema - leftGroup := joinExpr.Children[0] - rightGroup := joinExpr.Children[1] - - switch join.JoinType { - case plannercore.LeftOuterJoin, plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin: - leftGroup = pushTopNDownOuterJoinToChild(topN, leftGroup) - case plannercore.RightOuterJoin: - rightGroup = pushTopNDownOuterJoinToChild(topN, rightGroup) - default: - return nil, false, false, nil - } - - newJoinExpr := memo.NewGroupExpr(join) - newJoinExpr.SetChildren(leftGroup, rightGroup) - newTopNExpr := memo.NewGroupExpr(topN) - newTopNExpr.SetChildren(memo.NewGroupWithSchema(newJoinExpr, joinSchema)) - newTopNExpr.AddAppliedRule(r) - return []*memo.GroupExpr{newTopNExpr}, true, false, nil -} - -// PushTopNDownProjection pushes TopN to Projection. -type PushTopNDownProjection struct { - baseRule -} - -// NewRulePushTopNDownProjection creates a new Transformation PushTopNDownProjection. -// The pattern of this rule is `TopN->Projection->X` to `Projection->TopN->X`. -func NewRulePushTopNDownProjection() Transformation { - rule := &PushTopNDownProjection{} - rule.pattern = memo.BuildPattern( - memo.OperandTopN, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandProjection, memo.EngineTiDBOnly), - ) - return rule -} - -// Match implements Transformation interface. -func (r *PushTopNDownProjection) Match(expr *memo.ExprIter) bool { - proj := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) - for _, expr := range proj.Exprs { - if expression.HasAssignSetVarFunc(expr) { - return false - } - } - return true -} - -// OnTransform implements Transformation interface. -// This rule tries to pushes the TopN through Projection. -func (r *PushTopNDownProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - topN := old.GetExpr().ExprNode.(*plannercore.LogicalTopN) - proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) - childGroup := old.Children[0].GetExpr().Children[0] - - newTopN := plannercore.LogicalTopN{ - Offset: topN.Offset, - Count: topN.Count, - }.Init(topN.SCtx(), topN.SelectBlockOffset()) - - newTopN.ByItems = make([]*util.ByItems, 0, len(topN.ByItems)) - for _, by := range topN.ByItems { - newTopN.ByItems = append(newTopN.ByItems, &util.ByItems{ - Expr: expression.ColumnSubstitute(by.Expr, old.Children[0].Group.Prop.Schema, proj.Exprs), - Desc: by.Desc, - }) - } - - // remove meaningless constant sort items. - for i := len(newTopN.ByItems) - 1; i >= 0; i-- { - switch newTopN.ByItems[i].Expr.(type) { - case *expression.Constant, *expression.CorrelatedColumn: - topN.ByItems = append(newTopN.ByItems[:i], newTopN.ByItems[i+1:]...) - } - } - projExpr := memo.NewGroupExpr(proj) - topNExpr := memo.NewGroupExpr(newTopN) - topNExpr.SetChildren(childGroup) - topNGroup := memo.NewGroupWithSchema(topNExpr, childGroup.Prop.Schema) - projExpr.SetChildren(topNGroup) - 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 -} - -// PushTopNDownTiKVSingleGather pushes the top-n down to child of TiKVSingleGather. -type PushTopNDownTiKVSingleGather struct { - baseRule -} - -// NewRulePushTopNDownTiKVSingleGather creates a new Transformation PushTopNDownTiKVSingleGather. -// The pattern of this rule is `TopN -> TiKVSingleGather`. -func NewRulePushTopNDownTiKVSingleGather() Transformation { - rule := &PushTopNDownTiKVSingleGather{} - rule.pattern = memo.BuildPattern( - memo.OperandTopN, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandTiKVSingleGather, memo.EngineTiDBOnly), - ) - return rule -} - -// Match implements Transformation interface. -// Use appliedRuleSet in GroupExpr to avoid re-apply rules. -func (r *PushTopNDownTiKVSingleGather) Match(expr *memo.ExprIter) bool { - return !expr.GetExpr().HasAppliedRule(r) -} - -// OnTransform implements Transformation interface. -// It transforms `TopN -> TiKVSingleGather` to `TopN(Final) -> TiKVSingleGather -> TopN(Partial)`. -func (r *PushTopNDownTiKVSingleGather) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - topN := old.GetExpr().ExprNode.(*plannercore.LogicalTopN) - topNSchema := old.Children[0].Group.Prop.Schema - gather := old.Children[0].GetExpr().ExprNode.(*plannercore.TiKVSingleGather) - childGroup := old.Children[0].GetExpr().Children[0] - - particalTopN := plannercore.LogicalTopN{ - ByItems: topN.ByItems, - Count: topN.Count + topN.Offset, - }.Init(topN.SCtx(), topN.SelectBlockOffset()) - partialTopNExpr := memo.NewGroupExpr(particalTopN) - partialTopNExpr.SetChildren(childGroup) - partialTopNGroup := memo.NewGroupWithSchema(partialTopNExpr, topNSchema).SetEngineType(childGroup.EngineType) - - gatherExpr := memo.NewGroupExpr(gather) - gatherExpr.SetChildren(partialTopNGroup) - gatherGroup := memo.NewGroupWithSchema(gatherExpr, topNSchema) - - finalTopNExpr := memo.NewGroupExpr(topN) - finalTopNExpr.SetChildren(gatherGroup) - finalTopNExpr.AddAppliedRule(r) - return []*memo.GroupExpr{finalTopNExpr}, true, false, nil -} - -// MergeAdjacentTopN merge adjacent TopN. -type MergeAdjacentTopN struct { - baseRule -} - -// NewRuleMergeAdjacentTopN creates a new Transformation MergeAdjacentTopN. -// The pattern of this rule is `TopN->TopN->X`. -func NewRuleMergeAdjacentTopN() Transformation { - rule := &MergeAdjacentTopN{} - rule.pattern = memo.BuildPattern( - memo.OperandTopN, - memo.EngineAll, - memo.NewPattern(memo.OperandTopN, memo.EngineAll), - ) - return rule -} - -// Match implements Transformation interface. -func (r *MergeAdjacentTopN) Match(expr *memo.ExprIter) bool { - topN := expr.GetExpr().ExprNode.(*plannercore.LogicalTopN) - child := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalTopN) - - // We can use this rule when the sort columns of parent TopN is a prefix of child TopN. - if len(child.ByItems) < len(topN.ByItems) { - return false - } - for i := 0; i < len(topN.ByItems); i++ { - if !topN.ByItems[i].Equal(topN.SCtx(), child.ByItems[i]) { - return false - } - } - return true -} - -// OnTransform implements Transformation interface. -// This rule tries to merge adjacent TopN. -func (r *MergeAdjacentTopN) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - topN := old.GetExpr().ExprNode.(*plannercore.LogicalTopN) - child := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalTopN) - childGroups := old.Children[0].GetExpr().Children - - if child.Count <= topN.Offset { - tableDual := plannercore.LogicalTableDual{RowCount: 0}.Init(child.SCtx(), child.SelectBlockOffset()) - tableDual.SetSchema(old.GetExpr().Schema()) - tableDualExpr := memo.NewGroupExpr(tableDual) - return []*memo.GroupExpr{tableDualExpr}, true, true, nil - } - - offset := child.Offset + topN.Offset - count := uint64(math.Min(float64(child.Count-topN.Offset), float64(topN.Count))) - newTopN := plannercore.LogicalTopN{ - Count: count, - Offset: offset, - ByItems: child.ByItems, - }.Init(child.SCtx(), child.SelectBlockOffset()) - newTopNExpr := memo.NewGroupExpr(newTopN) - newTopNExpr.SetChildren(childGroups...) - 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. -type MergeAggregationProjection struct { - baseRule -} - -// NewRuleMergeAggregationProjection creates a new Transformation MergeAggregationProjection. -// The pattern of this rule is: `Aggregation -> Projection`. -func NewRuleMergeAggregationProjection() Transformation { - rule := &MergeAggregationProjection{} - rule.pattern = memo.BuildPattern( - memo.OperandAggregation, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandProjection, memo.EngineTiDBOnly), - ) - return rule -} - -// Match implements Transformation interface. -func (r *MergeAggregationProjection) Match(old *memo.ExprIter) bool { - proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) - if plannercore.ExprsHasSideEffects(proj.Exprs) { - return false - } - return true -} - -// OnTransform implements Transformation interface. -// It will transform `Aggregation->Projection->X` to `Aggregation->X`. -func (r *MergeAggregationProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - oldAgg := old.GetExpr().ExprNode.(*plannercore.LogicalAggregation) - proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) - projSchema := old.Children[0].GetExpr().Schema() - - groupByItems := make([]expression.Expression, len(oldAgg.GroupByItems)) - for i, item := range oldAgg.GroupByItems { - groupByItems[i] = expression.ColumnSubstitute(item, projSchema, proj.Exprs) - } - - aggFuncs := make([]*aggregation.AggFuncDesc, len(oldAgg.AggFuncs)) - for i, aggFunc := range oldAgg.AggFuncs { - aggFuncs[i] = aggFunc.Clone() - newArgs := make([]expression.Expression, len(aggFunc.Args)) - for j, arg := range aggFunc.Args { - newArgs[j] = expression.ColumnSubstitute(arg, projSchema, proj.Exprs) - } - aggFuncs[i].Args = newArgs - } - - newAgg := plannercore.LogicalAggregation{ - GroupByItems: groupByItems, - AggFuncs: aggFuncs, - }.Init(oldAgg.SCtx(), oldAgg.SelectBlockOffset()) - - newAggExpr := memo.NewGroupExpr(newAgg) - newAggExpr.SetChildren(old.Children[0].GetExpr().Children...) - return []*memo.GroupExpr{newAggExpr}, false, 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 - } - - // If there is only one aggFunc, we don't need to guarantee that the child of it is a data - // source, or whether the sort can be eliminated. This transformation won't be worse than previous. - // Make sure that the aggFunc are Max or Min. - // TODO: If there have only one Max or Min aggFunc and the other aggFuncs are FirstRow() can also use this rule. Waiting for the not null prop is maintained. - if len(agg.AggFuncs) != 1 { - return false - } - if agg.AggFuncs[0].Name != ast.AggFuncMax && agg.AggFuncs[0].Name != ast.AggFuncMin { - 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() - f := agg.AggFuncs[0] - - // 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[0])) > 0 { - // If it can be NULL, we need to filter NULL out first. - if !mysql.HasNotNullFlag(f.Args[0].GetType().Flag) { - sel := plannercore.LogicalSelection{}.Init(ctx, agg.SelectBlockOffset()) - isNullFunc := expression.NewFunctionInternal(ctx, ast.IsNull, types.NewFieldType(mysql.TypeTiny), f.Args[0]) - 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 []*util.ByItems - byItems = append(byItems, &util.ByItems{ - Expr: f.Args[0], - 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 -} - -// NewRuleMergeAdjacentSelection creates a new Transformation MergeAdjacentSelection. -// The pattern of this rule is `Selection->Selection->X`. -func NewRuleMergeAdjacentSelection() Transformation { - rule := &MergeAdjacentSelection{} - rule.pattern = memo.BuildPattern( - memo.OperandSelection, - memo.EngineAll, - memo.NewPattern(memo.OperandSelection, memo.EngineAll), - ) - return rule -} - -// OnTransform implements Transformation interface. -// This rule tries to merge adjacent selection, with no simplification. -func (r *MergeAdjacentSelection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) - child := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalSelection) - childGroups := old.Children[0].GetExpr().Children - - conditions := make([]expression.Expression, 0, len(sel.Conditions)+len(child.Conditions)) - conditions = append(conditions, sel.Conditions...) - conditions = append(conditions, child.Conditions...) - newSel := plannercore.LogicalSelection{Conditions: conditions}.Init(sel.SCtx(), sel.SelectBlockOffset()) - newSelExpr := memo.NewGroupExpr(newSel) - newSelExpr.SetChildren(childGroups...) - return []*memo.GroupExpr{newSelExpr}, true, false, nil -} - -// MergeAdjacentLimit merge the adjacent limit. -type MergeAdjacentLimit struct { - baseRule -} - -// NewRuleMergeAdjacentLimit creates a new Transformation MergeAdjacentLimit. -// The pattern of this rule is `Limit->Limit->X`. -func NewRuleMergeAdjacentLimit() Transformation { - rule := &MergeAdjacentLimit{} - rule.pattern = memo.BuildPattern( - memo.OperandLimit, - memo.EngineAll, - memo.NewPattern(memo.OperandLimit, memo.EngineAll), - ) - return rule -} - -// OnTransform implements Transformation interface. -// This rule tries to merge adjacent limit. -func (r *MergeAdjacentLimit) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - limit := old.GetExpr().ExprNode.(*plannercore.LogicalLimit) - child := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalLimit) - childGroups := old.Children[0].GetExpr().Children - - if child.Count <= limit.Offset { - tableDual := plannercore.LogicalTableDual{RowCount: 0}.Init(child.SCtx(), child.SelectBlockOffset()) - tableDual.SetSchema(old.GetExpr().Schema()) - tableDualExpr := memo.NewGroupExpr(tableDual) - return []*memo.GroupExpr{tableDualExpr}, true, true, nil - } - - offset := child.Offset + limit.Offset - count := uint64(math.Min(float64(child.Count-limit.Offset), float64(limit.Count))) - newLimit := plannercore.LogicalLimit{ - Offset: offset, - Count: count, - }.Init(limit.SCtx(), limit.SelectBlockOffset()) - newLimitExpr := memo.NewGroupExpr(newLimit) - newLimitExpr.SetChildren(childGroups...) - return []*memo.GroupExpr{newLimitExpr}, true, false, nil -} - -// TransformLimitToTableDual convert limit to TableDual. -type TransformLimitToTableDual struct { - baseRule -} - -// NewRuleTransformLimitToTableDual creates a new Transformation TransformLimitToTableDual. -// The pattern of this rule is `Limit->X`. -func NewRuleTransformLimitToTableDual() Transformation { - rule := &TransformLimitToTableDual{} - rule.pattern = memo.BuildPattern( - memo.OperandLimit, - memo.EngineAll, - ) - return rule -} - -// Match implements Transformation interface. -func (r *TransformLimitToTableDual) Match(expr *memo.ExprIter) bool { - limit := expr.GetExpr().ExprNode.(*plannercore.LogicalLimit) - return 0 == limit.Count -} - -// OnTransform implements Transformation interface. -// This rule tries to convert limit to tableDual. -func (r *TransformLimitToTableDual) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - limit := old.GetExpr().ExprNode.(*plannercore.LogicalLimit) - tableDual := plannercore.LogicalTableDual{RowCount: 0}.Init(limit.SCtx(), limit.SelectBlockOffset()) - tableDual.SetSchema(old.GetExpr().Schema()) - tableDualExpr := memo.NewGroupExpr(tableDual) - return []*memo.GroupExpr{tableDualExpr}, true, true, nil -} - -// PushLimitDownOuterJoin pushes Limit through Join. -type PushLimitDownOuterJoin struct { - baseRule -} - -// NewRulePushLimitDownOuterJoin creates a new Transformation PushLimitDownOuterJoin. -// The pattern of this rule is `Limit -> Join`. -func NewRulePushLimitDownOuterJoin() Transformation { - rule := &PushLimitDownOuterJoin{} - rule.pattern = memo.BuildPattern( - memo.OperandLimit, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandJoin, memo.EngineTiDBOnly), - ) - return rule -} - -// Match implements Transformation interface. -func (r *PushLimitDownOuterJoin) Match(expr *memo.ExprIter) bool { - if expr.GetExpr().HasAppliedRule(r) { - return false - } - join := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin) - return join.JoinType.IsOuterJoin() -} - -// OnTransform implements Transformation interface. -// This rule tries to pushes the Limit through outer Join. -func (r *PushLimitDownOuterJoin) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - limit := old.GetExpr().ExprNode.(*plannercore.LogicalLimit) - join := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin) - joinSchema := old.Children[0].Group.Prop.Schema - leftGroup := old.Children[0].GetExpr().Children[0] - rightGroup := old.Children[0].GetExpr().Children[1] - - switch join.JoinType { - case plannercore.LeftOuterJoin, plannercore.LeftOuterSemiJoin, plannercore.AntiLeftOuterSemiJoin: - leftGroup = r.pushLimitDownOuterJoinToChild(limit, leftGroup) - case plannercore.RightOuterJoin: - rightGroup = r.pushLimitDownOuterJoinToChild(limit, rightGroup) - default: - return nil, false, false, nil - } - - newJoinExpr := memo.NewGroupExpr(join) - newJoinExpr.SetChildren(leftGroup, rightGroup) - newLimitExpr := memo.NewGroupExpr(limit) - newLimitExpr.SetChildren(memo.NewGroupWithSchema(newJoinExpr, joinSchema)) - newLimitExpr.AddAppliedRule(r) - return []*memo.GroupExpr{newLimitExpr}, true, false, nil -} - -func (r *PushLimitDownOuterJoin) pushLimitDownOuterJoinToChild(limit *plannercore.LogicalLimit, outerGroup *memo.Group) *memo.Group { - newLimit := plannercore.LogicalLimit{ - Count: limit.Count + limit.Offset, - }.Init(limit.SCtx(), limit.SelectBlockOffset()) - newLimitGroup := memo.NewGroupExpr(newLimit) - newLimitGroup.SetChildren(outerGroup) - return memo.NewGroupWithSchema(newLimitGroup, outerGroup.Prop.Schema) -} - -// PushLimitDownTiKVSingleGather pushes the limit down to child of TiKVSingleGather. -type PushLimitDownTiKVSingleGather struct { - baseRule -} - -// NewRulePushLimitDownTiKVSingleGather creates a new Transformation PushLimitDownTiKVSingleGather. -// The pattern of this rule is `Limit -> TiKVSingleGather`. -func NewRulePushLimitDownTiKVSingleGather() Transformation { - rule := &PushLimitDownTiKVSingleGather{} - rule.pattern = memo.BuildPattern( - memo.OperandLimit, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandTiKVSingleGather, memo.EngineTiDBOnly), - ) - return rule -} - -// Match implements Transformation interface. -// Use appliedRuleSet in GroupExpr to avoid re-apply rules. -func (r *PushLimitDownTiKVSingleGather) Match(expr *memo.ExprIter) bool { - return !expr.GetExpr().HasAppliedRule(r) -} - -// OnTransform implements Transformation interface. -// It transforms `Limit -> TiKVSingleGather` to `Limit(Final) -> TiKVSingleGather -> Limit(Partial)`. -func (r *PushLimitDownTiKVSingleGather) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - limit := old.GetExpr().ExprNode.(*plannercore.LogicalLimit) - limitSchema := old.Children[0].Group.Prop.Schema - gather := old.Children[0].GetExpr().ExprNode.(*plannercore.TiKVSingleGather) - childGroup := old.Children[0].GetExpr().Children[0] - - particalLimit := plannercore.LogicalLimit{ - Count: limit.Count + limit.Offset, - }.Init(limit.SCtx(), limit.SelectBlockOffset()) - partialLimitExpr := memo.NewGroupExpr(particalLimit) - partialLimitExpr.SetChildren(childGroup) - partialLimitGroup := memo.NewGroupWithSchema(partialLimitExpr, limitSchema).SetEngineType(childGroup.EngineType) - - gatherExpr := memo.NewGroupExpr(gather) - gatherExpr.SetChildren(partialLimitGroup) - gatherGroup := memo.NewGroupWithSchema(gatherExpr, limitSchema) - - finalLimitExpr := memo.NewGroupExpr(limit) - finalLimitExpr.SetChildren(gatherGroup) - finalLimitExpr.AddAppliedRule(r) - return []*memo.GroupExpr{finalLimitExpr}, true, false, nil -} - -type outerJoinEliminator struct { -} - -func (*outerJoinEliminator) prepareForEliminateOuterJoin(joinExpr *memo.GroupExpr) (ok bool, innerChildIdx int, outerGroup *memo.Group, innerGroup *memo.Group, outerUniqueIDs set.Int64Set) { - join := joinExpr.ExprNode.(*plannercore.LogicalJoin) - - switch join.JoinType { - case plannercore.LeftOuterJoin: - innerChildIdx = 1 - case plannercore.RightOuterJoin: - innerChildIdx = 0 - default: - ok = false - return - } - outerGroup = joinExpr.Children[1^innerChildIdx] - innerGroup = joinExpr.Children[innerChildIdx] - - outerUniqueIDs = set.NewInt64Set() - for _, outerCol := range outerGroup.Prop.Schema.Columns { - outerUniqueIDs.Insert(outerCol.UniqueID) - } - - ok = true - return -} - -// check whether one of unique keys sets is contained by inner join keys. -func (*outerJoinEliminator) isInnerJoinKeysContainUniqueKey(innerGroup *memo.Group, joinKeys *expression.Schema) (bool, error) { - // builds UniqueKey info of innerGroup. - innerGroup.BuildKeyInfo() - for _, keyInfo := range innerGroup.Prop.Schema.Keys { - joinKeysContainKeyInfo := true - for _, col := range keyInfo { - if !joinKeys.Contains(col) { - joinKeysContainKeyInfo = false - break - } - } - if joinKeysContainKeyInfo { - return true, nil - } - } - return false, nil -} - -// EliminateOuterJoinBelowAggregation eliminate the outer join which below aggregation. -type EliminateOuterJoinBelowAggregation struct { - baseRule - outerJoinEliminator -} - -// NewRuleEliminateOuterJoinBelowAggregation creates a new Transformation EliminateOuterJoinBelowAggregation. -// The pattern of this rule is `Aggregation->Join->X`. -func NewRuleEliminateOuterJoinBelowAggregation() Transformation { - rule := &EliminateOuterJoinBelowAggregation{} - rule.pattern = memo.BuildPattern( - memo.OperandAggregation, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandJoin, memo.EngineTiDBOnly), - ) - return rule -} - -// Match implements Transformation interface. -func (r *EliminateOuterJoinBelowAggregation) Match(expr *memo.ExprIter) bool { - joinType := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin).JoinType - return joinType == plannercore.LeftOuterJoin || joinType == plannercore.RightOuterJoin -} - -// OnTransform implements Transformation interface. -// This rule tries to eliminate outer join which below aggregation. -func (r *EliminateOuterJoinBelowAggregation) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - agg := old.GetExpr().ExprNode.(*plannercore.LogicalAggregation) - joinExpr := old.Children[0].GetExpr() - join := joinExpr.ExprNode.(*plannercore.LogicalJoin) - - ok, innerChildIdx, outerGroup, innerGroup, outerUniqueIDs := r.prepareForEliminateOuterJoin(joinExpr) - if !ok { - return nil, false, false, nil - } - - // only when agg only use the columns from outer table can eliminate outer join. - if !plannercore.IsColsAllFromOuterTable(agg.GetUsedCols(), outerUniqueIDs) { - return nil, false, false, nil - } - // outer join elimination with duplicate agnostic aggregate functions. - _, aggCols := plannercore.GetDupAgnosticAggCols(agg, nil) - if len(aggCols) > 0 { - newAggExpr := memo.NewGroupExpr(agg) - newAggExpr.SetChildren(outerGroup) - return []*memo.GroupExpr{newAggExpr}, true, false, nil - } - // outer join elimination without duplicate agnostic aggregate functions. - innerJoinKeys := join.ExtractJoinKeys(innerChildIdx) - contain, err := r.isInnerJoinKeysContainUniqueKey(innerGroup, innerJoinKeys) - if err != nil { - return nil, false, false, err - } - if contain { - newAggExpr := memo.NewGroupExpr(agg) - newAggExpr.SetChildren(outerGroup) - return []*memo.GroupExpr{newAggExpr}, true, false, nil - } - - return nil, false, false, nil -} - -// EliminateOuterJoinBelowProjection eliminate the outer join which below projection. -type EliminateOuterJoinBelowProjection struct { - baseRule - outerJoinEliminator -} - -// NewRuleEliminateOuterJoinBelowProjection creates a new Transformation EliminateOuterJoinBelowProjection. -// The pattern of this rule is `Projection->Join->X`. -func NewRuleEliminateOuterJoinBelowProjection() Transformation { - rule := &EliminateOuterJoinBelowProjection{} - rule.pattern = memo.BuildPattern( - memo.OperandProjection, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandJoin, memo.EngineTiDBOnly), - ) - return rule -} - -// Match implements Transformation interface. -func (r *EliminateOuterJoinBelowProjection) Match(expr *memo.ExprIter) bool { - joinType := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalJoin).JoinType - return joinType == plannercore.LeftOuterJoin || joinType == plannercore.RightOuterJoin -} - -// OnTransform implements Transformation interface. -// This rule tries to eliminate outer join which below projection. -func (r *EliminateOuterJoinBelowProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - proj := old.GetExpr().ExprNode.(*plannercore.LogicalProjection) - joinExpr := old.Children[0].GetExpr() - join := joinExpr.ExprNode.(*plannercore.LogicalJoin) - - ok, innerChildIdx, outerGroup, innerGroup, outerUniqueIDs := r.prepareForEliminateOuterJoin(joinExpr) - if !ok { - return nil, false, false, nil - } - - // only when proj only use the columns from outer table can eliminate outer join. - if !plannercore.IsColsAllFromOuterTable(proj.GetUsedCols(), outerUniqueIDs) { - return nil, false, false, nil - } - - innerJoinKeys := join.ExtractJoinKeys(innerChildIdx) - contain, err := r.isInnerJoinKeysContainUniqueKey(innerGroup, innerJoinKeys) - if err != nil { - return nil, false, false, err - } - if contain { - newProjExpr := memo.NewGroupExpr(proj) - newProjExpr.SetChildren(outerGroup) - return []*memo.GroupExpr{newProjExpr}, true, false, nil - } - - return nil, false, false, nil -} - -// TransformAggregateCaseToSelection convert Agg(case when) to Agg->Selection. -type TransformAggregateCaseToSelection struct { - baseRule -} - -// NewRuleTransformAggregateCaseToSelection creates a new Transformation TransformAggregateCaseToSelection. -// The pattern of this rule is `Agg->X`. -func NewRuleTransformAggregateCaseToSelection() Transformation { - rule := &TransformAggregateCaseToSelection{} - rule.pattern = memo.BuildPattern( - memo.OperandAggregation, - memo.EngineTiDBOnly, - ) - return rule -} - -// Match implements Transformation interface. -func (r *TransformAggregateCaseToSelection) Match(expr *memo.ExprIter) bool { - agg := expr.GetExpr().ExprNode.(*plannercore.LogicalAggregation) - return agg.IsCompleteModeAgg() && len(agg.GroupByItems) == 0 && len(agg.AggFuncs) == 1 && len(agg.AggFuncs[0].Args) == 1 && r.isTwoOrThreeArgCase(agg.AggFuncs[0].Args[0]) -} - -// OnTransform implements Transformation interface. -// This rule tries to convert Agg(case when) to Agg->Selection. -func (r *TransformAggregateCaseToSelection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - agg := old.GetExpr().ExprNode.(*plannercore.LogicalAggregation) - - ok, newConditions, newAggFuncs := r.transform(agg) - if !ok { - return nil, false, false, nil - } - - newSel := plannercore.LogicalSelection{Conditions: newConditions}.Init(agg.SCtx(), agg.SelectBlockOffset()) - newSelExpr := memo.NewGroupExpr(newSel) - newSelExpr.SetChildren(old.GetExpr().Children...) - newSelGroup := memo.NewGroupWithSchema(newSelExpr, old.GetExpr().Children[0].Prop.Schema) - - newAgg := plannercore.LogicalAggregation{ - AggFuncs: newAggFuncs, - GroupByItems: agg.GroupByItems, - }.Init(agg.SCtx(), agg.SelectBlockOffset()) - newAgg.CopyAggHints(agg) - newAggExpr := memo.NewGroupExpr(newAgg) - newAggExpr.SetChildren(newSelGroup) - return []*memo.GroupExpr{newAggExpr}, true, false, nil -} - -func (r *TransformAggregateCaseToSelection) transform(agg *plannercore.LogicalAggregation) (ok bool, newConditions []expression.Expression, newAggFuncs []*aggregation.AggFuncDesc) { - aggFuncDesc := agg.AggFuncs[0] - aggFuncName := aggFuncDesc.Name - ctx := agg.SCtx() - - caseFunc := aggFuncDesc.Args[0].(*expression.ScalarFunction) - conditionFromCase := caseFunc.GetArgs()[0] - caseArgs := caseFunc.GetArgs() - caseArgsNum := len(caseArgs) - - // `case when a>0 then null else a end` should be converted to `case when !(a>0) then a else null end`. - var nullFlip = caseArgsNum == 3 && caseArgs[1].Equal(ctx, expression.NewNull()) && !caseArgs[2].Equal(ctx, expression.NewNull()) - // `case when a>0 then 0 else a end` should be converted to `case when !(a>0) then a else 0 end`. - var zeroFlip = !nullFlip && caseArgsNum == 3 && caseArgs[1].Equal(ctx, expression.NewZero()) - - var outputIdx int - if nullFlip || zeroFlip { - outputIdx = 2 - newConditions = []expression.Expression{expression.NewFunctionInternal(ctx, ast.UnaryNot, types.NewFieldType(mysql.TypeTiny), conditionFromCase)} - } else { - outputIdx = 1 - newConditions = expression.SplitCNFItems(conditionFromCase) - } - - if aggFuncDesc.HasDistinct { - // Just one style supported: - // COUNT(DISTINCT CASE WHEN x = 'foo' THEN y END) - // => - // newAggFuncDesc: COUNT(DISTINCT y), newCondition: x = 'foo' - - if aggFuncName == ast.AggFuncCount && r.isOnlyOneNotNull(ctx, caseArgs, caseArgsNum, outputIdx) { - newAggFuncDesc := aggFuncDesc.Clone() - newAggFuncDesc.Args = []expression.Expression{caseArgs[outputIdx]} - return true, newConditions, []*aggregation.AggFuncDesc{newAggFuncDesc} - } - return false, nil, nil - } - - // Two styles supported: - // - // A1: AGG(CASE WHEN x = 'foo' THEN cnt END) - // => newAggFuncDesc: AGG(cnt), newCondition: x = 'foo' - // A2: SUM(CASE WHEN x = 'foo' THEN cnt ELSE 0 END) - // => newAggFuncDesc: SUM(cnt), newCondition: x = 'foo' - - switch { - case r.allowsSelection(aggFuncName) && (caseArgsNum == 2 || caseArgs[3-outputIdx].Equal(ctx, expression.NewNull())), // Case A1 - aggFuncName == ast.AggFuncSum && caseArgsNum == 3 && caseArgs[3-outputIdx].Equal(ctx, expression.NewZero()): // Case A2 - newAggFuncDesc := aggFuncDesc.Clone() - newAggFuncDesc.Args = []expression.Expression{caseArgs[outputIdx]} - return true, newConditions, []*aggregation.AggFuncDesc{newAggFuncDesc} - default: - return false, nil, nil - } -} - -func (r *TransformAggregateCaseToSelection) allowsSelection(aggFuncName string) bool { - return aggFuncName != ast.AggFuncFirstRow -} - -func (r *TransformAggregateCaseToSelection) isOnlyOneNotNull(ctx sessionctx.Context, args []expression.Expression, argsNum int, outputIdx int) bool { - return !args[outputIdx].Equal(ctx, expression.NewNull()) && (argsNum == 2 || args[3-outputIdx].Equal(ctx, expression.NewNull())) -} - -// TransformAggregateCaseToSelection only support `case when cond then var end` and `case when cond then var1 else var2 end`. -func (r *TransformAggregateCaseToSelection) isTwoOrThreeArgCase(expr expression.Expression) bool { - scalarFunc, ok := expr.(*expression.ScalarFunction) - if !ok { - return false - } - return scalarFunc.FuncName.L == ast.Case && (len(scalarFunc.GetArgs()) == 2 || len(scalarFunc.GetArgs()) == 3) -} - -// TransformAggToProj convert Agg to Proj. -type TransformAggToProj struct { - baseRule -} - -// NewRuleTransformAggToProj creates a new Transformation TransformAggToProj. -// The pattern of this rule is `Agg`. -func NewRuleTransformAggToProj() Transformation { - rule := &TransformAggToProj{} - rule.pattern = memo.BuildPattern( - memo.OperandAggregation, - memo.EngineTiDBOnly, - ) - return rule -} - -// Match implements Transformation interface. -func (r *TransformAggToProj) Match(expr *memo.ExprIter) bool { - agg := expr.GetExpr().ExprNode.(*plannercore.LogicalAggregation) - - if !agg.IsCompleteModeAgg() { - return false - } - - for _, af := range agg.AggFuncs { - // TODO(issue #9968): same as rule_aggregation_elimination.go -> tryToEliminateAggregation. - // waiting for (issue #14616): `nullable` information. - if af.Name == ast.AggFuncGroupConcat { - return false - } - } - - childGroup := expr.GetExpr().Children[0] - childGroup.BuildKeyInfo() - schemaByGroupby := expression.NewSchema(agg.GetGroupByCols()...) - for _, key := range childGroup.Prop.Schema.Keys { - if schemaByGroupby.ColumnsIndices(key) != nil { - return true - } - } - - return false -} - -// OnTransform implements Transformation interface. -// This rule tries to convert agg to proj. -func (r *TransformAggToProj) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - agg := old.GetExpr().ExprNode.(*plannercore.LogicalAggregation) - if ok, proj := plannercore.ConvertAggToProj(agg, old.GetExpr().Schema()); ok { - newProjExpr := memo.NewGroupExpr(proj) - newProjExpr.SetChildren(old.GetExpr().Children...) - return []*memo.GroupExpr{newProjExpr}, true, false, nil - } - - return nil, false, false, nil -} - -// InjectProjectionBelowTopN injects two Projections below and upon TopN if TopN's ByItems -// contain ScalarFunctions. -type InjectProjectionBelowTopN struct { - baseRule -} - -// NewRuleInjectProjectionBelowTopN creates a new Transformation InjectProjectionBelowTopN. -// It will extract the ScalarFunctions of `ByItems` into a Projection and injects it below TopN. -// When a Projection is injected as the child of TopN, we need to add another Projection upon -// TopN to prune the extra Columns. -// The reason why we need this rule is that, TopNExecutor in TiDB does not support ScalarFunction -// as `ByItem`. So we have to use a Projection to calculate the ScalarFunctions in advance. -// The pattern of this rule is: a single TopN -func NewRuleInjectProjectionBelowTopN() Transformation { - rule := &InjectProjectionBelowTopN{} - rule.pattern = memo.BuildPattern( - memo.OperandTopN, - memo.EngineTiDBOnly, - ) - return rule -} - -// Match implements Transformation interface. -func (r *InjectProjectionBelowTopN) Match(expr *memo.ExprIter) bool { - topN := expr.GetExpr().ExprNode.(*plannercore.LogicalTopN) - for _, item := range topN.ByItems { - if _, ok := item.Expr.(*expression.ScalarFunction); ok { - return true - } - } - return false -} - -// OnTransform implements Transformation interface. -// It will convert `TopN -> X` to `Projection -> TopN -> Projection -> X`. -func (r *InjectProjectionBelowTopN) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - topN := old.GetExpr().ExprNode.(*plannercore.LogicalTopN) - oldTopNSchema := old.GetExpr().Schema() - - // Construct top Projection. - topProjExprs := make([]expression.Expression, oldTopNSchema.Len()) - for i := range oldTopNSchema.Columns { - topProjExprs[i] = oldTopNSchema.Columns[i] - } - topProj := plannercore.LogicalProjection{ - Exprs: topProjExprs, - }.Init(topN.SCtx(), topN.SelectBlockOffset()) - topProj.SetSchema(oldTopNSchema) - - // Construct bottom Projection. - bottomProjExprs := make([]expression.Expression, 0, oldTopNSchema.Len()+len(topN.ByItems)) - bottomProjSchema := make([]*expression.Column, 0, oldTopNSchema.Len()+len(topN.ByItems)) - for _, col := range oldTopNSchema.Columns { - bottomProjExprs = append(bottomProjExprs, col) - bottomProjSchema = append(bottomProjSchema, col) - } - newByItems := make([]*util.ByItems, 0, len(topN.ByItems)) - for _, item := range topN.ByItems { - itemExpr := item.Expr - if _, isScalarFunc := itemExpr.(*expression.ScalarFunction); !isScalarFunc { - newByItems = append(newByItems, item) - continue - } - bottomProjExprs = append(bottomProjExprs, itemExpr) - newCol := &expression.Column{ - UniqueID: topN.SCtx().GetSessionVars().AllocPlanColumnID(), - RetType: itemExpr.GetType(), - } - bottomProjSchema = append(bottomProjSchema, newCol) - newByItems = append(newByItems, &util.ByItems{Expr: newCol, Desc: item.Desc}) - } - bottomProj := plannercore.LogicalProjection{ - Exprs: bottomProjExprs, - }.Init(topN.SCtx(), topN.SelectBlockOffset()) - newSchema := expression.NewSchema(bottomProjSchema...) - bottomProj.SetSchema(newSchema) - - newTopN := plannercore.LogicalTopN{ - ByItems: newByItems, - Offset: topN.Offset, - Count: topN.Count, - }.Init(topN.SCtx(), topN.SelectBlockOffset()) - - // Construct GroupExpr, Group (TopProj -> TopN -> BottomProj -> Child) - bottomProjGroupExpr := memo.NewGroupExpr(bottomProj) - bottomProjGroupExpr.SetChildren(old.GetExpr().Children[0]) - bottomProjGroup := memo.NewGroupWithSchema(bottomProjGroupExpr, newSchema) - - topNGroupExpr := memo.NewGroupExpr(newTopN) - topNGroupExpr.SetChildren(bottomProjGroup) - topNGroup := memo.NewGroupWithSchema(topNGroupExpr, newSchema) - - topProjGroupExpr := memo.NewGroupExpr(topProj) - topProjGroupExpr.SetChildren(topNGroup) - return []*memo.GroupExpr{topProjGroupExpr}, true, false, nil -} - -// TransformApplyToJoin transforms a LogicalApply to LogicalJoin if it's -// inner children has no correlated columns from it's outer schema. -type TransformApplyToJoin struct { - baseRule -} - -// NewRuleTransformApplyToJoin creates a new Transformation TransformApplyToJoin. -// The pattern of this rule is: `Apply -> (X, Y)`. -func NewRuleTransformApplyToJoin() Transformation { - rule := &TransformApplyToJoin{} - rule.pattern = memo.NewPattern(memo.OperandApply, memo.EngineTiDBOnly) - return rule -} - -// OnTransform implements Transformation interface. -func (r *TransformApplyToJoin) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - apply := old.GetExpr().ExprNode.(*plannercore.LogicalApply) - groupExpr := old.GetExpr() - // It's safe to use the old apply instead of creating a new LogicalApply here, - // Because apply.CorCols will only be used and updated by this rule during Transformation. - apply.CorCols = r.extractCorColumnsBySchema(groupExpr.Children[1], groupExpr.Children[0].Prop.Schema) - if len(apply.CorCols) != 0 { - return nil, false, false, nil - } - - join := apply.LogicalJoin.Shallow() - joinGroupExpr := memo.NewGroupExpr(join) - joinGroupExpr.SetChildren(groupExpr.Children...) - return []*memo.GroupExpr{joinGroupExpr}, true, false, nil -} - -func (r *TransformApplyToJoin) extractCorColumnsBySchema(innerGroup *memo.Group, outerSchema *expression.Schema) []*expression.CorrelatedColumn { - corCols := r.extractCorColumnsFromGroup(innerGroup) - return plannercore.ExtractCorColumnsBySchema(corCols, outerSchema) -} - -func (r *TransformApplyToJoin) extractCorColumnsFromGroup(g *memo.Group) []*expression.CorrelatedColumn { - corCols := make([]*expression.CorrelatedColumn, 0) - for elem := g.Equivalents.Front(); elem != nil; elem = elem.Next() { - expr := elem.Value.(*memo.GroupExpr) - corCols = append(corCols, expr.ExprNode.ExtractCorrelatedCols()...) - for _, child := range expr.Children { - corCols = append(corCols, r.extractCorColumnsFromGroup(child)...) - } - } - // We may have duplicate CorrelatedColumns here, but it won't influence - // the logic of the transformation. Apply.CorCols will be deduplicated in - // `ResolveIndices`. - return corCols -} - -// PullSelectionUpApply pulls up the inner-side Selection into Apply as -// its join condition. -type PullSelectionUpApply struct { - baseRule -} - -// NewRulePullSelectionUpApply creates a new Transformation PullSelectionUpApply. -// The pattern of this rule is: `Apply -> (Any, Selection)`. -func NewRulePullSelectionUpApply() Transformation { - rule := &PullSelectionUpApply{} - rule.pattern = memo.BuildPattern( - memo.OperandApply, - memo.EngineTiDBOnly, - memo.NewPattern(memo.OperandAny, memo.EngineTiDBOnly), // outer child - memo.NewPattern(memo.OperandSelection, memo.EngineTiDBOnly), // inner child - ) - return rule -} - -// OnTransform implements Transformation interface. -// This rule tries to pull up the inner side Selection, and add these conditions -// to Join condition inside the Apply. -func (r *PullSelectionUpApply) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { - apply := old.GetExpr().ExprNode.(*plannercore.LogicalApply) - outerChildGroup := old.Children[0].Group - innerChildGroup := old.Children[1].Group - sel := old.Children[1].GetExpr().ExprNode.(*plannercore.LogicalSelection) - newConds := make([]expression.Expression, 0, len(sel.Conditions)) - for _, cond := range sel.Conditions { - newConds = append(newConds, cond.Clone().Decorrelate(outerChildGroup.Prop.Schema)) - } - newApply := plannercore.LogicalApply{ - LogicalJoin: *(apply.LogicalJoin.Shallow()), - CorCols: apply.CorCols, - }.Init(apply.SCtx(), apply.SelectBlockOffset()) - // Update Join conditions. - eq, left, right, other := newApply.LogicalJoin.ExtractOnCondition(newConds, outerChildGroup.Prop.Schema, innerChildGroup.Prop.Schema, false, false) - newApply.LogicalJoin.AppendJoinConds(eq, left, right, other) - - newApplyGroupExpr := memo.NewGroupExpr(newApply) - newApplyGroupExpr.SetChildren(outerChildGroup, old.Children[1].GetExpr().Children[0]) - return []*memo.GroupExpr{newApplyGroupExpr}, false, false, nil ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 3217a31568fa7..9e719ca437685 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1120,13 +1120,8 @@ func (lt *LogicalTopN) getPhysLimits() []PhysicalPlan { return ret } -<<<<<<< HEAD // Check if this prop's columns can match by items totally. func matchItems(p *property.PhysicalProperty, items []*ByItems) bool { -======= -// MatchItems checks if this prop's columns can match by items totally. -func MatchItems(p *property.PhysicalProperty, items []*util.ByItems) bool { ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) if len(items) < len(p.Items) { return false } diff --git a/planner/core/explain.go b/planner/core/explain.go index 537ecab2f4c95..b4461bdef0a19 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" - "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/statistics" ) @@ -530,212 +529,7 @@ func formatWindowFuncDescs(buffer *bytes.Buffer, descs []*aggregation.WindowFunc if i != 0 { buffer.WriteString(", ") } -<<<<<<< HEAD buffer.WriteString(desc.String()) } return buffer } -======= - fmt.Fprintf(buffer, "%v->%v", desc, schema.Columns[winFuncStartIdx+i]) - } - return buffer -} - -// ExplainInfo implements Plan interface. -func (p *LogicalJoin) ExplainInfo() string { - buffer := bytes.NewBufferString(p.JoinType.String()) - if len(p.EqualConditions) > 0 { - fmt.Fprintf(buffer, ", equal:%v", p.EqualConditions) - } - if len(p.LeftConditions) > 0 { - fmt.Fprintf(buffer, ", left cond:%s", - expression.SortedExplainExpressionList(p.LeftConditions)) - } - if len(p.RightConditions) > 0 { - fmt.Fprintf(buffer, ", right cond:%s", - expression.SortedExplainExpressionList(p.RightConditions)) - } - if len(p.OtherConditions) > 0 { - fmt.Fprintf(buffer, ", other cond:%s", - expression.SortedExplainExpressionList(p.OtherConditions)) - } - return buffer.String() -} - -// ExplainInfo implements Plan interface. -func (p *LogicalAggregation) ExplainInfo() string { - buffer := bytes.NewBufferString("") - if len(p.GroupByItems) > 0 { - fmt.Fprintf(buffer, "group by:%s, ", - expression.SortedExplainExpressionList(p.GroupByItems)) - } - if len(p.AggFuncs) > 0 { - buffer.WriteString("funcs:") - for i, agg := range p.AggFuncs { - buffer.WriteString(aggregation.ExplainAggFunc(agg)) - if i+1 < len(p.AggFuncs) { - buffer.WriteString(", ") - } - } - } - return buffer.String() -} - -// ExplainInfo implements Plan interface. -func (p *LogicalProjection) ExplainInfo() string { - return expression.ExplainExpressionList(p.Exprs, p.schema) -} - -// ExplainInfo implements Plan interface. -func (p *LogicalSelection) ExplainInfo() string { - return string(expression.SortedExplainExpressionList(p.Conditions)) -} - -// ExplainInfo implements Plan interface. -func (p *LogicalApply) ExplainInfo() string { - return p.LogicalJoin.ExplainInfo() -} - -// ExplainInfo implements Plan interface. -func (p *LogicalTableDual) ExplainInfo() string { - return fmt.Sprintf("rowcount:%d", p.RowCount) -} - -// ExplainInfo implements Plan interface. -func (p *DataSource) ExplainInfo() string { - buffer := bytes.NewBufferString("") - tblName := p.tableInfo.Name.O - if p.TableAsName != nil && p.TableAsName.O != "" { - tblName = p.TableAsName.O - } - fmt.Fprintf(buffer, "table:%s", tblName) - if p.isPartition { - if pi := p.tableInfo.GetPartitionInfo(); pi != nil { - partitionName := pi.GetNameByID(p.physicalTableID) - fmt.Fprintf(buffer, ", partition:%s", partitionName) - } - } - return buffer.String() -} - -// ExplainInfo implements Plan interface. -func (p *LogicalUnionScan) ExplainInfo() string { - buffer := bytes.NewBufferString("") - fmt.Fprintf(buffer, "conds:%s", - expression.SortedExplainExpressionList(p.conditions)) - fmt.Fprintf(buffer, ", handle:%s", p.handleCol.ExplainInfo()) - return buffer.String() -} - -func explainByItems(buffer *bytes.Buffer, byItems []*util.ByItems) *bytes.Buffer { - for i, item := range byItems { - order := "asc" - if item.Desc { - order = "desc" - } - fmt.Fprintf(buffer, "%s:%s", item.Expr.ExplainInfo(), order) - if i+1 < len(byItems) { - buffer.WriteString(", ") - } - } - return buffer -} - -func explainNormalizedByItems(buffer *bytes.Buffer, byItems []*util.ByItems) *bytes.Buffer { - for i, item := range byItems { - order := "asc" - if item.Desc { - order = "desc" - } - fmt.Fprintf(buffer, "%s:%s", item.Expr.ExplainNormalizedInfo(), order) - if i+1 < len(byItems) { - buffer.WriteString(", ") - } - } - return buffer -} - -// ExplainInfo implements Plan interface. -func (p *LogicalSort) ExplainInfo() string { - buffer := bytes.NewBufferString("") - return explainByItems(buffer, p.ByItems).String() -} - -// ExplainInfo implements Plan interface. -func (p *LogicalTopN) ExplainInfo() string { - buffer := bytes.NewBufferString("") - buffer = explainByItems(buffer, p.ByItems) - fmt.Fprintf(buffer, ", offset:%v, count:%v", p.Offset, p.Count) - return buffer.String() -} - -// ExplainInfo implements Plan interface. -func (p *LogicalLimit) ExplainInfo() string { - return fmt.Sprintf("offset:%v, count:%v", p.Offset, p.Count) -} - -// ExplainInfo implements Plan interface. -func (p *LogicalTableScan) ExplainInfo() string { - buffer := bytes.NewBufferString(p.Source.ExplainInfo()) - if p.Source.handleCol != nil { - fmt.Fprintf(buffer, ", pk col:%s", p.Source.handleCol.ExplainInfo()) - } - if len(p.AccessConds) > 0 { - fmt.Fprintf(buffer, ", cond:%v", p.AccessConds) - } - return buffer.String() -} - -// ExplainInfo implements Plan interface. -func (p *LogicalIndexScan) ExplainInfo() string { - buffer := bytes.NewBufferString(p.Source.ExplainInfo()) - index := p.Index - if len(index.Columns) > 0 { - buffer.WriteString(", index:") - for i, idxCol := range index.Columns { - buffer.WriteString(idxCol.Name.O) - if i+1 < len(index.Columns) { - buffer.WriteString(", ") - } - } - } - if len(p.AccessConds) > 0 { - fmt.Fprintf(buffer, ", cond:%v", p.AccessConds) - } - return buffer.String() -} - -// ExplainInfo implements Plan interface. -func (p *TiKVSingleGather) ExplainInfo() string { - buffer := bytes.NewBufferString(p.Source.ExplainInfo()) - if p.IsIndexGather { - buffer.WriteString(", index:" + p.Index.Name.String()) - } - return buffer.String() -} - -// MetricTableTimeFormat is the time format for metric table explain and format. -const MetricTableTimeFormat = "2006-01-02 15:04:05.999" - -// ExplainInfo implements Plan interface. -func (p *PhysicalMemTable) ExplainInfo() string { - accessObject, operatorInfo := p.AccessObject(), p.OperatorInfo(false) - if len(operatorInfo) == 0 { - return accessObject - } - return accessObject + ", " + operatorInfo -} - -// AccessObject implements dataAccesser interface. -func (p *PhysicalMemTable) AccessObject() string { - return "table:" + p.Table.Name.O -} - -// OperatorInfo implements dataAccesser interface. -func (p *PhysicalMemTable) OperatorInfo(_ bool) string { - if p.Extractor != nil { - return p.Extractor.explainInfo(p) - } - return "" -} ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index bb5808eddbb51..bef7316ba2aa3 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" @@ -53,11 +54,7 @@ var invalidTask = &rootTask{cst: math.MaxFloat64} // getPropByOrderByItems will check if this sort property can be pushed or not. In order to simplify the problem, we only // consider the case that all expression are columns. -<<<<<<< HEAD -func getPropByOrderByItems(items []*ByItems) (*property.PhysicalProperty, bool) { -======= -func GetPropByOrderByItems(items []*util.ByItems) (*property.PhysicalProperty, bool) { ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) +func getPropByOrderByItems(items []*util.ByItems) (*property.PhysicalProperty, bool) { propItems := make([]property.Item, 0, len(items)) for _, item := range items { col, ok := item.Expr.(*expression.Column) @@ -69,32 +66,6 @@ func GetPropByOrderByItems(items []*util.ByItems) (*property.PhysicalProperty, b return &property.PhysicalProperty{Items: propItems}, true } -<<<<<<< HEAD -======= -// GetPropByOrderByItemsContainScalarFunc will check if this sort property can be pushed or not. In order to simplify the -// problem, we only consider the case that all expression are columns or some special scalar functions. -func GetPropByOrderByItemsContainScalarFunc(items []*util.ByItems) (*property.PhysicalProperty, bool, bool) { - propItems := make([]property.Item, 0, len(items)) - onlyColumn := true - for _, item := range items { - switch expr := item.Expr.(type) { - case *expression.Column: - propItems = append(propItems, property.Item{Col: expr, Desc: item.Desc}) - case *expression.ScalarFunction: - col, desc := expr.GetSingleColumn(item.Desc) - if col == nil { - return nil, false, false - } - propItems = append(propItems, property.Item{Col: col, Desc: desc}) - onlyColumn = false - default: - return nil, false, false - } - } - return &property.PhysicalProperty{Items: propItems}, true, onlyColumn -} - ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty) (task, error) { if !prop.IsEmpty() { return invalidTask, nil diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index bc14f82529d21..5025334fa9a0e 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -1030,28 +1030,6 @@ func (b *PlanBuilder) buildUnionAll(ctx context.Context, subPlan []LogicalPlan) return u } -<<<<<<< HEAD -// ByItems wraps a "by" item. -type ByItems struct { - Expr expression.Expression - Desc bool -} - -// String implements fmt.Stringer interface. -func (by *ByItems) String() string { - if by.Desc { - return fmt.Sprintf("%s true", by.Expr) - } - return by.Expr.String() -} - -// Clone makes a copy of ByItems. -func (by *ByItems) Clone() *ByItems { - return &ByItems{Expr: by.Expr.Clone(), Desc: by.Desc} -} - -======= ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) // itemTransformer transforms ParamMarkerExpr to PositionExpr in the context of ByItem type itemTransformer struct { } @@ -1075,13 +1053,8 @@ func (b *PlanBuilder) buildSort(ctx context.Context, p LogicalPlan, byItems []*a } else { b.curClause = orderByClause } -<<<<<<< HEAD sort := LogicalSort{}.Init(b.ctx) - exprs := make([]*ByItems, 0, len(byItems)) -======= - sort := LogicalSort{}.Init(b.ctx, b.getSelectOffset()) exprs := make([]*util.ByItems, 0, len(byItems)) ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) transformer := &itemTransformer{} for _, item := range byItems { newExpr, _ := item.Expr.Accept(transformer) diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 2e886f1cf071b..8b80ddb058960 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -2678,20 +2678,6 @@ func (s *testPlanSuite) TestSkylinePruning(c *C) { case *LogicalSort: byItems = v.ByItems lp = lp.Children()[0] -<<<<<<< HEAD -======= - case *LogicalProjection: - newItems := make([]*util.ByItems, 0, len(byItems)) - for _, col := range byItems { - idx := v.schema.ColumnIndex(col.Expr.(*expression.Column)) - switch expr := v.Exprs[idx].(type) { - case *expression.Column: - newItems = append(newItems, &util.ByItems{Expr: expr, Desc: col.Desc}) - } - } - byItems = newItems - lp = lp.Children()[0] ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) default: lp = lp.Children()[0] } diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index b5f06dae49f06..29b2a53dd347e 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" diff --git a/planner/core/pb_to_plan.go b/planner/core/pb_to_plan.go deleted file mode 100644 index 5ab1e76062b6f..0000000000000 --- a/planner/core/pb_to_plan.go +++ /dev/null @@ -1,277 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package core - -import ( - "strings" - - "github.com/pingcap/errors" - "github.com/pingcap/parser/model" - "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/expression/aggregation" - "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/planner/util" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tipb/go-tipb" -) - -// PBPlanBuilder uses to build physical plan from dag protocol buffers. -type PBPlanBuilder struct { - sctx sessionctx.Context - tps []*types.FieldType - is infoschema.InfoSchema -} - -// NewPBPlanBuilder creates a new pb plan builder. -func NewPBPlanBuilder(sctx sessionctx.Context, is infoschema.InfoSchema) *PBPlanBuilder { - return &PBPlanBuilder{sctx: sctx, is: is} -} - -// Build builds physical plan from dag protocol buffers. -func (b *PBPlanBuilder) Build(executors []*tipb.Executor) (p PhysicalPlan, err error) { - var src PhysicalPlan - for i := 0; i < len(executors); i++ { - curr, err := b.pbToPhysicalPlan(executors[i]) - if err != nil { - return nil, errors.Trace(err) - } - curr.SetChildren(src) - src = curr - } - _, src = b.predicatePushDown(src, nil) - return src, nil -} - -func (b *PBPlanBuilder) pbToPhysicalPlan(e *tipb.Executor) (p PhysicalPlan, err error) { - switch e.Tp { - case tipb.ExecType_TypeTableScan: - p, err = b.pbToTableScan(e) - case tipb.ExecType_TypeSelection: - p, err = b.pbToSelection(e) - case tipb.ExecType_TypeTopN: - p, err = b.pbToTopN(e) - case tipb.ExecType_TypeLimit: - p, err = b.pbToLimit(e) - case tipb.ExecType_TypeAggregation: - p, err = b.pbToAgg(e, false) - case tipb.ExecType_TypeStreamAgg: - p, err = b.pbToAgg(e, true) - default: - // TODO: Support other types. - err = errors.Errorf("this exec type %v doesn't support yet.", e.GetTp()) - } - return p, err -} - -func (b *PBPlanBuilder) pbToTableScan(e *tipb.Executor) (PhysicalPlan, error) { - tblScan := e.TblScan - tbl, ok := b.is.TableByID(tblScan.TableId) - if !ok { - return nil, infoschema.ErrTableNotExists.GenWithStack("Table which ID = %d does not exist.", tblScan.TableId) - } - dbInfo, ok := b.is.SchemaByTable(tbl.Meta()) - if !ok { - return nil, infoschema.ErrDatabaseNotExists.GenWithStack("Database of table ID = %d does not exist.", tblScan.TableId) - } - // Currently only support cluster table. - if !tbl.Type().IsClusterTable() { - return nil, errors.Errorf("table %s is not a cluster table", tbl.Meta().Name.L) - } - columns, err := b.convertColumnInfo(tbl.Meta(), tblScan.Columns) - if err != nil { - return nil, err - } - schema := b.buildTableScanSchema(tbl.Meta(), columns) - p := PhysicalMemTable{ - DBName: dbInfo.Name, - Table: tbl.Meta(), - Columns: columns, - }.Init(b.sctx, nil, 0) - p.SetSchema(schema) - if strings.ToUpper(p.Table.Name.O) == infoschema.ClusterTableSlowLog { - p.Extractor = &SlowQueryExtractor{} - } - return p, nil -} - -func (b *PBPlanBuilder) buildTableScanSchema(tblInfo *model.TableInfo, columns []*model.ColumnInfo) *expression.Schema { - schema := expression.NewSchema(make([]*expression.Column, 0, len(columns))...) - for _, col := range tblInfo.Columns { - for _, colInfo := range columns { - if col.ID != colInfo.ID { - continue - } - newCol := &expression.Column{ - UniqueID: b.sctx.GetSessionVars().AllocPlanColumnID(), - ID: col.ID, - RetType: &col.FieldType, - } - schema.Append(newCol) - } - } - return schema -} - -func (b *PBPlanBuilder) pbToSelection(e *tipb.Executor) (PhysicalPlan, error) { - conds, err := expression.PBToExprs(e.Selection.Conditions, b.tps, b.sctx.GetSessionVars().StmtCtx) - if err != nil { - return nil, err - } - p := PhysicalSelection{ - Conditions: conds, - }.Init(b.sctx, nil, 0) - return p, nil -} - -func (b *PBPlanBuilder) pbToTopN(e *tipb.Executor) (PhysicalPlan, error) { - topN := e.TopN - sc := b.sctx.GetSessionVars().StmtCtx - byItems := make([]*util.ByItems, 0, len(topN.OrderBy)) - for _, item := range topN.OrderBy { - expr, err := expression.PBToExpr(item.Expr, b.tps, sc) - if err != nil { - return nil, errors.Trace(err) - } - byItems = append(byItems, &util.ByItems{Expr: expr, Desc: item.Desc}) - } - p := PhysicalTopN{ - ByItems: byItems, - Count: topN.Limit, - }.Init(b.sctx, nil, 0) - return p, nil -} - -func (b *PBPlanBuilder) pbToLimit(e *tipb.Executor) (PhysicalPlan, error) { - p := PhysicalLimit{ - Count: e.Limit.Limit, - }.Init(b.sctx, nil, 0) - return p, nil -} - -func (b *PBPlanBuilder) pbToAgg(e *tipb.Executor, isStreamAgg bool) (PhysicalPlan, error) { - aggFuncs, groupBys, err := b.getAggInfo(e) - if err != nil { - return nil, errors.Trace(err) - } - schema := b.buildAggSchema(aggFuncs, groupBys) - baseAgg := basePhysicalAgg{ - AggFuncs: aggFuncs, - GroupByItems: groupBys, - } - baseAgg.schema = schema - var partialAgg PhysicalPlan - if isStreamAgg { - partialAgg = baseAgg.initForStream(b.sctx, nil, 0) - } else { - partialAgg = baseAgg.initForHash(b.sctx, nil, 0) - } - return partialAgg, nil -} - -func (b *PBPlanBuilder) buildAggSchema(aggFuncs []*aggregation.AggFuncDesc, groupBys []expression.Expression) *expression.Schema { - schema := expression.NewSchema(make([]*expression.Column, 0, len(aggFuncs)+len(groupBys))...) - for _, agg := range aggFuncs { - newCol := &expression.Column{ - UniqueID: b.sctx.GetSessionVars().AllocPlanColumnID(), - RetType: agg.RetTp, - } - schema.Append(newCol) - } - return schema -} - -func (b *PBPlanBuilder) getAggInfo(executor *tipb.Executor) ([]*aggregation.AggFuncDesc, []expression.Expression, error) { - var err error - aggFuncs := make([]*aggregation.AggFuncDesc, 0, len(executor.Aggregation.AggFunc)) - for _, expr := range executor.Aggregation.AggFunc { - aggFunc, err := aggregation.PBExprToAggFuncDesc(b.sctx, expr, b.tps) - if err != nil { - return nil, nil, errors.Trace(err) - } - aggFuncs = append(aggFuncs, aggFunc) - } - groupBys, err := expression.PBToExprs(executor.Aggregation.GetGroupBy(), b.tps, b.sctx.GetSessionVars().StmtCtx) - if err != nil { - return nil, nil, errors.Trace(err) - } - return aggFuncs, groupBys, nil -} - -func (b *PBPlanBuilder) convertColumnInfo(tblInfo *model.TableInfo, pbColumns []*tipb.ColumnInfo) ([]*model.ColumnInfo, error) { - columns := make([]*model.ColumnInfo, 0, len(pbColumns)) - tps := make([]*types.FieldType, 0, len(pbColumns)) - for _, col := range pbColumns { - found := false - for _, colInfo := range tblInfo.Columns { - if col.ColumnId == colInfo.ID { - columns = append(columns, colInfo) - tps = append(tps, colInfo.FieldType.Clone()) - found = true - break - } - } - if !found { - return nil, errors.Errorf("Column ID %v of table %v not found", col.ColumnId, tblInfo.Name.L) - } - } - b.tps = tps - return columns, nil -} - -func (b *PBPlanBuilder) predicatePushDown(p PhysicalPlan, predicates []expression.Expression) ([]expression.Expression, PhysicalPlan) { - if p == nil { - return predicates, p - } - switch p.(type) { - case *PhysicalMemTable: - memTable := p.(*PhysicalMemTable) - if memTable.Extractor == nil { - return predicates, p - } - names := make([]*types.FieldName, 0, len(memTable.Columns)) - for _, col := range memTable.Columns { - names = append(names, &types.FieldName{ - TblName: memTable.Table.Name, - ColName: col.Name, - OrigTblName: memTable.Table.Name, - OrigColName: col.Name, - }) - } - // Set the expression column unique ID. - // Since the expression is build from PB, It has not set the expression column ID yet. - schemaCols := memTable.schema.Columns - cols := expression.ExtractColumnsFromExpressions([]*expression.Column{}, predicates, nil) - for i := range cols { - cols[i].UniqueID = schemaCols[cols[i].Index].UniqueID - } - predicates = memTable.Extractor.Extract(b.sctx, memTable.schema, names, predicates) - return predicates, memTable - case *PhysicalSelection: - selection := p.(*PhysicalSelection) - conditions, child := b.predicatePushDown(p.Children()[0], selection.Conditions) - if len(conditions) > 0 { - selection.Conditions = conditions - selection.SetChildren(child) - return predicates, selection - } - return predicates, child - default: - if children := p.Children(); len(children) > 0 { - _, child := b.predicatePushDown(children[0], nil) - p.SetChildren(child) - } - return predicates, p - } -} diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index db1643d669d51..f0d5a44e437d7 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -659,762 +659,5 @@ func (s *testPlanSuite) TestIndexJoinHint(c *C) { c.Assert(warnings[0].Level, Equals, stmtctx.WarnLevelWarning) c.Assert(warnings[0].Err.Error(), Equals, output[i].Warning) } -<<<<<<< HEAD -======= - c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) - } -} - -func (s *testPlanSuite) TestAggregationHints(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - - sessionVars := se.(sessionctx.Context).GetSessionVars() - sessionVars.HashAggFinalConcurrency = 1 - sessionVars.HashAggPartialConcurrency = 1 - - var input []struct { - SQL string - AggPushDown bool - } - var output []struct { - SQL string - Best string - Warning string - } - s.testData.GetTestCases(c, &input, &output) - ctx := context.Background() - for i, test := range input { - comment := Commentf("case:%v sql:%s", i, test) - se.GetSessionVars().StmtCtx.SetWarnings(nil) - se.GetSessionVars().AllowAggPushDown = test.AggPushDown - - stmt, err := s.ParseOneStmt(test.SQL, "", "") - c.Assert(err, IsNil, comment) - - p, _, err := planner.Optimize(ctx, se, stmt, s.is) - c.Assert(err, IsNil) - warnings := se.GetSessionVars().StmtCtx.GetWarnings() - - s.testData.OnRecord(func() { - output[i].SQL = test.SQL - output[i].Best = core.ToString(p) - if len(warnings) > 0 { - output[i].Warning = warnings[0].Err.Error() - } - }) - c.Assert(core.ToString(p), Equals, output[i].Best, comment) - if output[i].Warning == "" { - c.Assert(len(warnings), Equals, 0, comment) - } else { - c.Assert(len(warnings), Equals, 1, comment) - c.Assert(warnings[0].Level, Equals, stmtctx.WarnLevelWarning, comment) - c.Assert(warnings[0].Err.Error(), Equals, output[i].Warning, comment) - } - } -} - -func (s *testPlanSuite) TestAggToCopHint(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists ta") - tk.MustExec("create table ta(a int, b int, index(a))") - - var ( - input []string - output []struct { - SQL string - Best string - Warning string - } - ) - s.testData.GetTestCases(c, &input, &output) - - ctx := context.Background() - is := domain.GetDomain(tk.Se).InfoSchema() - for i, test := range input { - comment := Commentf("case:%v sql:%s", i, test) - s.testData.OnRecord(func() { - output[i].SQL = test - }) - c.Assert(test, Equals, output[i].SQL, comment) - - tk.Se.GetSessionVars().StmtCtx.SetWarnings(nil) - - stmt, err := s.ParseOneStmt(test, "", "") - c.Assert(err, IsNil, comment) - - p, _, err := planner.Optimize(ctx, tk.Se, stmt, is) - c.Assert(err, IsNil) - planString := core.ToString(p) - s.testData.OnRecord(func() { - output[i].Best = planString - }) - c.Assert(planString, Equals, output[i].Best, comment) - - warnings := tk.Se.GetSessionVars().StmtCtx.GetWarnings() - s.testData.OnRecord(func() { - if len(warnings) > 0 { - output[i].Warning = warnings[0].Err.Error() - } - }) - if output[i].Warning == "" { - c.Assert(len(warnings), Equals, 0, comment) - } else { - c.Assert(len(warnings), Equals, 1, comment) - c.Assert(warnings[0].Level, Equals, stmtctx.WarnLevelWarning, comment) - c.Assert(warnings[0].Err.Error(), Equals, output[i].Warning, comment) - } - } -} - -func (s *testPlanSuite) TestPushdownDistinctEnable(c *C) { - defer testleak.AfterTest(c)() - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - } - ) - s.testData.GetTestCases(c, &input, &output) - vars := []string{ - fmt.Sprintf("set @@session.%s = 1", variable.TiDBOptDistinctAggPushDown), - "set session tidb_opt_agg_push_down = 1", - } - s.doTestPushdownDistinct(c, vars, input, output) -} - -func (s *testPlanSuite) TestPushdownDistinctDisable(c *C) { - defer testleak.AfterTest(c)() - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - } - ) - s.testData.GetTestCases(c, &input, &output) - vars := []string{ - fmt.Sprintf("set @@session.%s = 0", variable.TiDBOptDistinctAggPushDown), - "set session tidb_opt_agg_push_down = 1", - } - s.doTestPushdownDistinct(c, vars, input, output) -} - -func (s *testPlanSuite) TestPushdownDistinctEnableAggPushDownDisable(c *C) { - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - } - ) - s.testData.GetTestCases(c, &input, &output) - vars := []string{ - fmt.Sprintf("set @@session.%s = 1", variable.TiDBOptDistinctAggPushDown), - "set session tidb_opt_agg_push_down = 0", - } - s.doTestPushdownDistinct(c, vars, input, output) -} - -func (s *testPlanSuite) doTestPushdownDistinct(c *C, vars, input []string, output []struct { - SQL string - Plan []string - Result []string -}) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) - tk.MustExec("use test") - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int, c int, index(c))") - tk.MustExec("insert into t values (1, 1, 1), (1, 1, 3), (1, 2, 3), (2, 1, 3), (1, 2, NULL);") - - tk.MustExec("drop table if exists pt") - tk.MustExec(`CREATE TABLE pt (a int, b int) PARTITION BY RANGE (a) ( - PARTITION p0 VALUES LESS THAN (2), - PARTITION p1 VALUES LESS THAN (100) - );`) - - tk.MustExec("drop table if exists ta") - tk.MustExec("create table ta(a int);") - tk.MustExec("insert into ta values(1), (1);") - tk.MustExec("drop table if exists tb") - tk.MustExec("create table tb(a int);") - tk.MustExec("insert into tb values(1), (1);") - - tk.MustExec("set session sql_mode=''") - tk.MustExec(fmt.Sprintf("set session %s=1", variable.TiDBHashAggPartialConcurrency)) - tk.MustExec(fmt.Sprintf("set session %s=1", variable.TiDBHashAggFinalConcurrency)) - - for _, v := range vars { - tk.MustExec(v) - } - - for i, ts := range input { - s.testData.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + ts).Rows()) - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) - }) - tk.MustQuery("explain " + ts).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) - } -} - -func (s *testPlanSuite) TestGroupConcatOrderby(c *C) { - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - } - ) - s.testData.GetTestCases(c, &input, &output) - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists test;") - tk.MustExec("create table test(id int, name int)") - tk.MustExec("insert into test values(1, 10);") - tk.MustExec("insert into test values(1, 20);") - tk.MustExec("insert into test values(1, 30);") - tk.MustExec("insert into test values(2, 20);") - tk.MustExec("insert into test values(3, 200);") - tk.MustExec("insert into test values(3, 500);") - - tk.MustExec("drop table if exists ptest;") - tk.MustExec("CREATE TABLE ptest (id int,name int) PARTITION BY RANGE ( id ) " + - "(PARTITION `p0` VALUES LESS THAN (2), PARTITION `p1` VALUES LESS THAN (11))") - tk.MustExec("insert into ptest select * from test;") - tk.MustExec(fmt.Sprintf("set session tidb_opt_distinct_agg_push_down = %v", 1)) - tk.MustExec(fmt.Sprintf("set session tidb_opt_agg_push_down = %v", 1)) - - for i, ts := range input { - s.testData.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + ts).Rows()) - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) - }) - tk.MustQuery("explain " + ts).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(ts).Check(testkit.Rows(output[i].Result...)) - } -} - -func (s *testPlanSuite) TestHintAlias(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - - tests := []struct { - sql1 string - sql2 string - }{ - { - sql1: "select /*+ TIDB_SMJ(t1) */ t1.a, t1.b from t t1, (select /*+ TIDB_INLJ(t3) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", - sql2: "select /*+ MERGE_JOIN(t1) */ t1.a, t1.b from t t1, (select /*+ INL_JOIN(t3) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", - }, - { - sql1: "select /*+ TIDB_HJ(t1) */ t1.a, t1.b from t t1, (select /*+ TIDB_SMJ(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", - sql2: "select /*+ HASH_JOIN(t1) */ t1.a, t1.b from t t1, (select /*+ MERGE_JOIN(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", - }, - { - sql1: "select /*+ TIDB_INLJ(t1) */ t1.a, t1.b from t t1, (select /*+ TIDB_HJ(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", - sql2: "select /*+ INL_JOIN(t1) */ t1.a, t1.b from t t1, (select /*+ HASH_JOIN(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", - }, - } - ctx := context.TODO() - for i, tt := range tests { - comment := Commentf("case:%v sql1:%s sql2:%s", i, tt.sql1, tt.sql2) - stmt1, err := s.ParseOneStmt(tt.sql1, "", "") - c.Assert(err, IsNil, comment) - stmt2, err := s.ParseOneStmt(tt.sql2, "", "") - c.Assert(err, IsNil, comment) - - p1, _, err := planner.Optimize(ctx, se, stmt1, s.is) - c.Assert(err, IsNil) - p2, _, err := planner.Optimize(ctx, se, stmt2, s.is) - c.Assert(err, IsNil) - - c.Assert(core.ToString(p1), Equals, core.ToString(p2)) - } -} - -func (s *testPlanSuite) TestIndexHint(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - - var input []string - var output []struct { - SQL string - Best string - HasWarn bool - Hints string - } - s.testData.GetTestCases(c, &input, &output) - ctx := context.Background() - for i, test := range input { - comment := Commentf("case:%v sql:%s", i, test) - se.GetSessionVars().StmtCtx.SetWarnings(nil) - - stmt, err := s.ParseOneStmt(test, "", "") - c.Assert(err, IsNil, comment) - - p, _, err := planner.Optimize(ctx, se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { - output[i].SQL = test - output[i].Best = core.ToString(p) - output[i].HasWarn = len(se.GetSessionVars().StmtCtx.GetWarnings()) > 0 - output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) - }) - c.Assert(core.ToString(p), Equals, output[i].Best, comment) - warnings := se.GetSessionVars().StmtCtx.GetWarnings() - if output[i].HasWarn { - c.Assert(warnings, HasLen, 1, comment) - } else { - c.Assert(warnings, HasLen, 0, comment) - } - c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) - } -} - -func (s *testPlanSuite) TestIndexMergeHint(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - - var input []string - var output []struct { - SQL string - Best string - HasWarn bool - Hints string - } - s.testData.GetTestCases(c, &input, &output) - ctx := context.Background() - for i, test := range input { - comment := Commentf("case:%v sql:%s", i, test) - se.GetSessionVars().StmtCtx.SetWarnings(nil) - stmt, err := s.ParseOneStmt(test, "", "") - c.Assert(err, IsNil, comment) - sctx := se.(sessionctx.Context) - err = executor.ResetContextOfStmt(sctx, stmt) - c.Assert(err, IsNil) - p, _, err := planner.Optimize(ctx, se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { - output[i].SQL = test - output[i].Best = core.ToString(p) - output[i].HasWarn = len(se.GetSessionVars().StmtCtx.GetWarnings()) > 0 - output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) - }) - c.Assert(core.ToString(p), Equals, output[i].Best, comment) - warnings := se.GetSessionVars().StmtCtx.GetWarnings() - if output[i].HasWarn { - c.Assert(warnings, HasLen, 1, comment) - } else { - c.Assert(warnings, HasLen, 0, comment) - } - c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) - } -} - -func (s *testPlanSuite) TestQueryBlockHint(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - - var input []string - var output []struct { - SQL string - Plan string - Hints string - } - s.testData.GetTestCases(c, &input, &output) - ctx := context.TODO() - for i, tt := range input { - comment := Commentf("case:%v sql: %s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) - - p, _, err := planner.Optimize(ctx, se, stmt, s.is) - c.Assert(err, IsNil, comment) - s.testData.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = core.ToString(p) - output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) - }) - c.Assert(core.ToString(p), Equals, output[i].Plan, comment) - c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) - } -} - -func (s *testPlanSuite) TestInlineProjection(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - ctx := context.Background() - _, err = se.Execute(ctx, "use test") - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `drop table if exists test.t1, test.t2;`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `create table test.t1(a bigint, b bigint, index idx_a(a), index idx_b(b));`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `create table test.t2(a bigint, b bigint, index idx_a(a), index idx_b(b));`) - c.Assert(err, IsNil) - - var input []string - var output []struct { - SQL string - Plan string - Hints string - } - is := domain.GetDomain(se).InfoSchema() - s.testData.GetTestCases(c, &input, &output) - for i, tt := range input { - comment := Commentf("case:%v sql: %s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) - - p, _, err := planner.Optimize(ctx, se, stmt, is) - c.Assert(err, IsNil, comment) - s.testData.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = core.ToString(p) - output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) - }) - c.Assert(core.ToString(p), Equals, output[i].Plan, comment) - c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) - } -} - -func (s *testPlanSuite) TestDAGPlanBuilderSplitAvg(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - tests := []struct { - sql string - plan string - }{ - { - sql: "select avg(a),avg(b),avg(c) from t", - plan: "TableReader(Table(t)->StreamAgg)->StreamAgg", - }, - { - sql: "select /*+ HASH_AGG() */ avg(a),avg(b),avg(c) from t", - plan: "TableReader(Table(t)->HashAgg)->HashAgg", - }, - } - - for _, tt := range tests { - comment := Commentf("for %s", tt.sql) - stmt, err := s.ParseOneStmt(tt.sql, "", "") - c.Assert(err, IsNil, comment) - - core.Preprocess(se, stmt, s.is) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil, comment) - - c.Assert(core.ToString(p), Equals, tt.plan, comment) - root, ok := p.(core.PhysicalPlan) - if !ok { - continue - } - testDAGPlanBuilderSplitAvg(c, root) - } -} - -func testDAGPlanBuilderSplitAvg(c *C, root core.PhysicalPlan) { - if p, ok := root.(*core.PhysicalTableReader); ok { - if p.TablePlans != nil { - baseAgg := p.TablePlans[len(p.TablePlans)-1] - if agg, ok := baseAgg.(*core.PhysicalHashAgg); ok { - for i, aggfunc := range agg.AggFuncs { - c.Assert(agg.Schema().Columns[i].RetType, Equals, aggfunc.RetTp) - } - } - if agg, ok := baseAgg.(*core.PhysicalStreamAgg); ok { - for i, aggfunc := range agg.AggFuncs { - c.Assert(agg.Schema().Columns[i].RetType, Equals, aggfunc.RetTp) - } - } - } - } - - childs := root.Children() - if childs == nil { - return - } - for _, son := range childs { - testDAGPlanBuilderSplitAvg(c, son) - } -} - -func (s *testPlanSuite) TestIndexJoinHint(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - ctx := context.Background() - _, err = se.Execute(ctx, "use test") - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `drop table if exists test.t1, test.t2, test.t;`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `create table test.t1(a bigint, b bigint, index idx_a(a), index idx_b(b));`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `create table test.t2(a bigint, b bigint, index idx_a(a), index idx_b(b));`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, "CREATE TABLE `t` ( `a` bigint(20) NOT NULL, `b` tinyint(1) DEFAULT NULL, `c` datetime DEFAULT NULL, `d` int(10) unsigned DEFAULT NULL, `e` varchar(20) DEFAULT NULL, `f` double DEFAULT NULL, `g` decimal(30,5) DEFAULT NULL, `h` float DEFAULT NULL, `i` date DEFAULT NULL, `j` timestamp NULL DEFAULT NULL, PRIMARY KEY (`a`), UNIQUE KEY `b` (`b`), KEY `c` (`c`,`d`,`e`), KEY `f` (`f`), KEY `g` (`g`,`h`), KEY `g_2` (`g`), UNIQUE KEY `g_3` (`g`), KEY `i` (`i`) );") - c.Assert(err, IsNil) - var input []string - var output []struct { - SQL string - Plan string - } - is := domain.GetDomain(se).InfoSchema() - s.testData.GetTestCases(c, &input, &output) - for i, tt := range input { - comment := Commentf("case:%v sql: %s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) - p, _, err := planner.Optimize(ctx, se, stmt, is) - c.Assert(err, IsNil, comment) - s.testData.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = core.ToString(p) - }) - c.Assert(core.ToString(p), Equals, output[i].Plan, comment) - } -} - -func (s *testPlanSuite) TestDAGPlanBuilderWindow(c *C) { - defer testleak.AfterTest(c)() - var input []string - var output []struct { - SQL string - Best string - } - s.testData.GetTestCases(c, &input, &output) - vars := []string{ - "set @@session.tidb_window_concurrency = 1", - } - s.doTestDAGPlanBuilderWindow(c, vars, input, output) -} - -func (s *testPlanSuite) TestDAGPlanBuilderWindowParallel(c *C) { - defer testleak.AfterTest(c)() - var input []string - var output []struct { - SQL string - Best string - } - s.testData.GetTestCases(c, &input, &output) - vars := []string{ - "set @@session.tidb_window_concurrency = 4", - } - s.doTestDAGPlanBuilderWindow(c, vars, input, output) -} - -func (s *testPlanSuite) doTestDAGPlanBuilderWindow(c *C, vars, input []string, output []struct { - SQL string - Best string -}) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - ctx := context.Background() - _, err = se.Execute(ctx, "use test") - c.Assert(err, IsNil) - - for _, v := range vars { - _, err = se.Execute(ctx, v) - c.Assert(err, IsNil) - } - - for i, tt := range input { - comment := Commentf("case:%v sql:%s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) - - err = se.NewTxn(context.Background()) - c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { - output[i].SQL = tt - output[i].Best = core.ToString(p) - }) - c.Assert(core.ToString(p), Equals, output[i].Best, comment) - } -} - -func (s *testPlanSuite) TestNominalSort(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - tk.MustExec("use test") - var input []string - var output []struct { - SQL string - Plan []string - Result []string - } - tk.MustExec("create table t (a int, b int, index idx_a(a), index idx_b(b))") - tk.MustExec("insert into t values(1, 1)") - tk.MustExec("insert into t values(1, 2)") - tk.MustExec("insert into t values(2, 4)") - tk.MustExec("insert into t values(3, 5)") - s.testData.GetTestCases(c, &input, &output) - for i, ts := range input { - s.testData.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + ts).Rows()) - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Rows()) - }) - tk.MustQuery("explain " + ts).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(ts).Check(testkit.Rows(output[i].Result...)) - } -} - -func (s *testPlanSuite) TestHintFromDiffDatabase(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - ctx := context.Background() - _, err = se.Execute(ctx, "use test") - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `drop table if exists test.t1`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `create table test.t1(a bigint, index idx_a(a));`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `create table test.t2(a bigint, index idx_a(a));`) - c.Assert(err, IsNil) - - _, err = se.Execute(ctx, "drop database if exists test2") - c.Assert(err, IsNil) - _, err = se.Execute(ctx, "create database test2") - c.Assert(err, IsNil) - _, err = se.Execute(ctx, "use test2") - c.Assert(err, IsNil) - - var input []string - var output []struct { - SQL string - Plan string - } - is := domain.GetDomain(se).InfoSchema() - s.testData.GetTestCases(c, &input, &output) - for i, tt := range input { - comment := Commentf("case:%v sql: %s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) - p, _, err := planner.Optimize(ctx, se, stmt, is) - c.Assert(err, IsNil, comment) - s.testData.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = core.ToString(p) - }) - c.Assert(core.ToString(p), Equals, output[i].Plan, comment) ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 8d54f8d1f7d36..d3a0f3b50acb8 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -19,11 +19,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/planner/property" -<<<<<<< HEAD -======= "github.com/pingcap/tidb/planner/util" - "github.com/pingcap/tidb/sessionctx" ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" @@ -365,15 +361,6 @@ type PhysicalSort struct { // appear in final physical operator tree. type NominalSort struct { basePhysicalPlan -<<<<<<< HEAD -======= - - // These two fields are used to switch ScalarFunctions to Constants. For these - // NominalSorts, we need to converted to Projections check if the ScalarFunctions - // are out of bounds. (issue #11653) - ByItems []*util.ByItems - OnlyColumn bool ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } // PhysicalUnionScan represents a union scan operator. diff --git a/planner/core/plan.go b/planner/core/plan.go index 79705ae6d251e..a3ccdc0914814 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -57,11 +57,7 @@ func enforceProperty(p *property.PhysicalProperty, tsk task, ctx sessionctx.Cont } tsk = finishCopTask(ctx, tsk) sortReqProp := &property.PhysicalProperty{TaskTp: property.RootTaskType, Items: p.Items, ExpectedCnt: math.MaxFloat64} -<<<<<<< HEAD - sort := PhysicalSort{ByItems: make([]*ByItems, 0, len(p.Items))}.Init(ctx, tsk.plan().statsInfo(), sortReqProp) -======= - sort := PhysicalSort{ByItems: make([]*util.ByItems, 0, len(p.Items))}.Init(ctx, tsk.plan().statsInfo(), tsk.plan().SelectBlockOffset(), sortReqProp) ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) + sort := PhysicalSort{ByItems: make([]*util.ByItems, 0, len(p.Items))}.Init(ctx, tsk.plan().statsInfo(), sortReqProp) for _, col := range p.Items { sort.ByItems = append(sort.ByItems, &util.ByItems{Expr: col.Col, Desc: col.Desc}) } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 9faf2912c380d..52d5530e08d63 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1377,23 +1377,12 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, } return substitutePlaceHolderDual(physical, p), nil } -<<<<<<< HEAD return p, nil } func substitutePlaceHolderDual(src PhysicalPlan, dst PhysicalPlan) PhysicalPlan { if dual, ok := src.(*PhysicalTableDual); ok && dual.placeHolder { return dst -======= - if show.Tp == ast.ShowVariables || show.Tp == ast.ShowStatus { - b.curClause = orderByClause - orderByCol := np.Schema().Columns[0].Clone().(*expression.Column) - sort := LogicalSort{ - ByItems: []*util.ByItems{{Expr: orderByCol}}, - }.Init(b.ctx, b.getSelectOffset()) - sort.SetChildren(np) - np = sort ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } for i, child := range src.Children() { newChild := substitutePlaceHolderDual(child, dst) diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 22d40545a1165..d3de39c5f8729 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -397,21 +397,17 @@ func (a *aggregationPushDownSolver) aggPushDown(p LogicalPlan) (_ LogicalPlan, e projChild := proj.children[0] agg.SetChildren(projChild) } else if union, ok1 := child.(*LogicalUnionAll); ok1 { -<<<<<<< HEAD + for _, aggFunc := range agg.AggFuncs { + if !a.isDecomposableWithUnion(aggFunc) { + return p, nil + } + } var gbyCols []*expression.Column gbyCols = expression.ExtractColumnsFromExpressions(gbyCols, agg.GroupByItems, nil) pushedAgg, err := a.makeNewAgg(agg.ctx, agg.AggFuncs, gbyCols) if err != nil { return nil, err } -======= - for _, aggFunc := range agg.AggFuncs { - if !a.isDecomposableWithUnion(aggFunc) { - return p, nil - } - } - pushedAgg := a.splitPartialAgg(agg) ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) newChildren := make([]LogicalPlan, 0, len(union.children)) for _, child := range union.children { newChild := a.pushAggCrossUnion(pushedAgg, union.Schema(), child) diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index d5aa70d06d225..c3e5338be1e52 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -23,11 +23,8 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" -<<<<<<< HEAD "github.com/pingcap/tidb/infoschema" -======= "github.com/pingcap/tidb/planner/util" ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) "github.com/pingcap/tidb/types" ) @@ -162,18 +159,10 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column) return child.PruneColumns(selfUsedCols) } -<<<<<<< HEAD -// PruneColumns implements LogicalPlan interface. -func (ls *LogicalSort) PruneColumns(parentUsedCols []*expression.Column) error { - child := ls.children[0] - for i := len(ls.ByItems) - 1; i >= 0; i-- { - cols := expression.ExtractColumns(ls.ByItems[i].Expr) -======= func pruneByItems(old []*util.ByItems) (new []*util.ByItems, parentUsedCols []*expression.Column) { new = make([]*util.ByItems, 0, len(old)) for _, byItem := range old { cols := expression.ExtractColumns(byItem.Expr) ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) if len(cols) == 0 { if !expression.IsRuntimeConstExpr(byItem.Expr) { new = append(new, byItem) @@ -200,23 +189,11 @@ func (ls *LogicalSort) PruneColumns(parentUsedCols []*expression.Column) error { } // PruneColumns implements LogicalPlan interface. -<<<<<<< HEAD func (p *LogicalUnionAll) PruneColumns(parentUsedCols []*expression.Column) error { used, err := getUsedList(parentUsedCols, p.schema) if err != nil { return err } -======= -// If any expression can view as a constant in execution stage, such as correlated column, constant, -// we do prune them. Note that we can't prune the expressions contain non-deterministic functions, such as rand(). -func (lt *LogicalTopN) PruneColumns(parentUsedCols []*expression.Column) error { - child := lt.children[0] - var cols []*expression.Column - lt.ByItems, cols = pruneByItems(lt.ByItems) - parentUsedCols = append(parentUsedCols, cols...) - return child.PruneColumns(parentUsedCols) -} ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) hasBeenUsed := false for i := range used { diff --git a/planner/core/rule_inject_extra_projection.go b/planner/core/rule_inject_extra_projection.go index 7101ef59b38cc..89fe281080104 100644 --- a/planner/core/rule_inject_extra_projection.go +++ b/planner/core/rule_inject_extra_projection.go @@ -165,11 +165,7 @@ func injectProjBelowAgg(aggPlan PhysicalPlan, aggFuncs []*aggregation.AggFuncDes // PhysicalTopN, some extra columns will be added into the schema of the // Projection, thus we need to add another Projection upon them to prune the // redundant columns. -<<<<<<< HEAD -func injectProjBelowSort(p PhysicalPlan, orderByItems []*ByItems) PhysicalPlan { -======= -func InjectProjBelowSort(p PhysicalPlan, orderByItems []*util.ByItems) PhysicalPlan { ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) +func injectProjBelowSort(p PhysicalPlan, orderByItems []*util.ByItems) PhysicalPlan { hasScalarFunc, numOrderByItems := false, len(orderByItems) for i := 0; !hasScalarFunc && i < numOrderByItems; i++ { _, isScalarFunc := orderByItems[i].Expr.(*expression.ScalarFunction) @@ -233,67 +229,3 @@ func InjectProjBelowSort(p PhysicalPlan, orderByItems []*util.ByItems) PhysicalP return topProj } -<<<<<<< HEAD -======= - -// TurnNominalSortIntoProj will turn nominal sort into two projections. This is to check if the scalar functions will -// overflow. -func TurnNominalSortIntoProj(p PhysicalPlan, onlyColumn bool, orderByItems []*util.ByItems) PhysicalPlan { - if onlyColumn { - return p.Children()[0] - } - - numOrderByItems := len(orderByItems) - childPlan := p.Children()[0] - - bottomProjSchemaCols := make([]*expression.Column, 0, len(childPlan.Schema().Columns)+numOrderByItems) - bottomProjExprs := make([]expression.Expression, 0, len(childPlan.Schema().Columns)+numOrderByItems) - for _, col := range childPlan.Schema().Columns { - newCol := col.Clone().(*expression.Column) - newCol.Index = childPlan.Schema().ColumnIndex(newCol) - bottomProjSchemaCols = append(bottomProjSchemaCols, newCol) - bottomProjExprs = append(bottomProjExprs, newCol) - } - - for _, item := range orderByItems { - itemExpr := item.Expr - if _, isScalarFunc := itemExpr.(*expression.ScalarFunction); !isScalarFunc { - continue - } - bottomProjExprs = append(bottomProjExprs, itemExpr) - newArg := &expression.Column{ - UniqueID: p.SCtx().GetSessionVars().AllocPlanColumnID(), - RetType: itemExpr.GetType(), - Index: len(bottomProjSchemaCols), - } - bottomProjSchemaCols = append(bottomProjSchemaCols, newArg) - } - - childProp := p.GetChildReqProps(0).Clone() - bottomProj := PhysicalProjection{ - Exprs: bottomProjExprs, - AvoidColumnEvaluator: false, - }.Init(p.SCtx(), childPlan.statsInfo().ScaleByExpectCnt(childProp.ExpectedCnt), p.SelectBlockOffset(), childProp) - bottomProj.SetSchema(expression.NewSchema(bottomProjSchemaCols...)) - bottomProj.SetChildren(childPlan) - - topProjExprs := make([]expression.Expression, 0, childPlan.Schema().Len()) - for i := range childPlan.Schema().Columns { - col := childPlan.Schema().Columns[i].Clone().(*expression.Column) - col.Index = i - topProjExprs = append(topProjExprs, col) - } - topProj := PhysicalProjection{ - Exprs: topProjExprs, - AvoidColumnEvaluator: false, - }.Init(p.SCtx(), childPlan.statsInfo().ScaleByExpectCnt(childProp.ExpectedCnt), p.SelectBlockOffset(), childProp) - topProj.SetSchema(childPlan.Schema().Clone()) - topProj.SetChildren(bottomProj) - - if origChildProj, isChildProj := childPlan.(*PhysicalProjection); isChildProj { - refine4NeighbourProj(bottomProj, origChildProj) - } - - return topProj -} ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) diff --git a/planner/core/rule_max_min_eliminate.go b/planner/core/rule_max_min_eliminate.go index d771639f61592..cb970a1516d4d 100644 --- a/planner/core/rule_max_min_eliminate.go +++ b/planner/core/rule_max_min_eliminate.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" ) @@ -179,13 +180,8 @@ func (a *maxMinEliminator) eliminateSingleMaxMin(agg *LogicalAggregation) *Logic // For max function, the sort order should be desc. desc := f.Name == ast.AggFuncMax // Compose Sort operator. -<<<<<<< HEAD sort := LogicalSort{}.Init(ctx) - sort.ByItems = append(sort.ByItems, &ByItems{f.Args[0], desc}) -======= - sort := LogicalSort{}.Init(ctx, agg.blockOffset) - sort.ByItems = append(sort.ByItems, &util.ByItems{Expr: f.Args[0], Desc: desc}) ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) + sort.ByItems = append(sort.ByItems, &util.ByItems{f.Args[0], desc}) sort.SetChildren(child) child = sort } diff --git a/planner/core/rule_topn_push_down.go b/planner/core/rule_topn_push_down.go index b6b0d8f45c45c..06cc238326d54 100644 --- a/planner/core/rule_topn_push_down.go +++ b/planner/core/rule_topn_push_down.go @@ -142,13 +142,8 @@ func (p *LogicalJoin) pushDownTopNToChild(topN *LogicalTopN, idx int) LogicalPla newTopN := LogicalTopN{ Count: topN.Count + topN.Offset, -<<<<<<< HEAD - ByItems: make([]*ByItems, len(topN.ByItems)), - }.Init(topN.ctx) -======= ByItems: make([]*util.ByItems, len(topN.ByItems)), - }.Init(topN.ctx, topN.blockOffset) ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) + }.Init(topN.ctx) for i := range topN.ByItems { newTopN.ByItems[i] = topN.ByItems[i].Clone() } diff --git a/planner/core/task.go b/planner/core/task.go index 722655da06f40..5613586801724 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -22,12 +22,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" -<<<<<<< HEAD -======= - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/plancodec" @@ -364,13 +359,8 @@ func (p *NominalSort) attach2Task(tasks ...task) task { return tasks[0] } -<<<<<<< HEAD func (p *PhysicalTopN) getPushedDownTopN() *PhysicalTopN { - newByItems := make([]*ByItems, 0, len(p.ByItems)) -======= -func (p *PhysicalTopN) getPushedDownTopN(childPlan PhysicalPlan) *PhysicalTopN { newByItems := make([]*util.ByItems, 0, len(p.ByItems)) ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) for _, expr := range p.ByItems { newByItems = append(newByItems, expr.Clone()) } diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/testdata/plan_suite_in.json index eab01a994194a..a6aaf2907db64 100644 --- a/planner/core/testdata/plan_suite_in.json +++ b/planner/core/testdata/plan_suite_in.json @@ -429,125 +429,5 @@ "cases": [ "select t1.a, (select count(t2.a) from t t2 where t2.g in (select t3.d from t t3 where t3.c = t1.a)) as agg_col from t t1;" ] -<<<<<<< HEAD -======= - }, - { - "name": "TestIndexJoinHint", - "cases": [ - "select /*+ INL_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;", - "select /*+ INL_HASH_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;", - "select /*+ INL_MERGE_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;", - // Issue 15484 - "select /*+ inl_merge_join(t2) */ t1.a, t2.a from t t1 left join t t2 use index(g_2) on t1.g=t2.g", - "select /*+inl_merge_join(t2)*/ t1.a, t2.a from t t1 left join t t2 use index(g_2) on t1.g=t2.g order by t1.a" - ] - }, - { - "name": "TestAggToCopHint", - "cases": [ - "select /*+ AGG_TO_COP(), HASH_AGG(), USE_INDEX(t) */ sum(a) from ta group by a", - "select /*+ AGG_TO_COP(), USE_INDEX(t) */ sum(b) from ta group by b", - "select /*+ AGG_TO_COP(), HASH_AGG(), USE_INDEX(t) */ distinct a from ta group by a", - "select /*+ AGG_TO_COP(), HASH_AGG(), HASH_JOIN(t1), USE_INDEX(t1), USE_INDEX(t2) */ sum(t1.a) from ta t1, ta t2 where t1.a = t2.b group by t1.a" - ] - }, - { - "name": "TestPushdownDistinctEnable", - "cases": [ - "select /*+ HASH_AGG() */ avg(distinct a) from t;", // InjectProjBelowAgg - "select /*+ HASH_AGG() */ a, count(distinct a) from t;", // firstrow(a) cannot be removed. - "select /*+ HASH_AGG() */ avg(b), c, avg(b), count(distinct A, B), count(distinct A), count(distinct c), sum(b) from t group by c;", - "select /*+ STREAM_AGG() */ count(distinct c) from t group by c;", // can push down - "select /*+ STREAM_AGG() */ count(distinct c) from t;", // can not push down because c is not in group by - "select /*+ HASH_AGG() */ count(distinct c) from t;", // can push down - "select count(distinct c) from t group by c;", - "select count(distinct c) from t;", // should not use streamAgg because c is not in group by - - "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", - "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;" - ] - }, - { - "name": "TestPushdownDistinctDisable", - "cases": [ - // do not pushdown even AGG_TO_COP is specified. - "select /*+ HASH_AGG(), AGG_TO_COP() */ avg(distinct a) from t;", - "select /*+ HASH_AGG(), AGG_TO_COP() */ a, count(distinct a) from t;", - "select /*+ HASH_AGG(), AGG_TO_COP() */ avg(b), c, avg(b), count(distinct A, B), count(distinct A), count(distinct c), sum(b) from t group by c;", - "select /*+ STREAM_AGG(), AGG_TO_COP() */ count(distinct c) from t group by c;", - "select /*+ STREAM_AGG(), AGG_TO_COP() */ count(distinct c) from t;", - "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct c) from t;", - "select /*+ AGG_TO_COP() */ count(distinct c) from t group by c;", - - "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", - "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;" - ] - }, - { - "name": "TestPushdownDistinctEnableAggPushDownDisable", - "cases": [ - "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", - "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;" - ] - }, - { - "name": "TestGroupConcatOrderby", - "cases": [ - "select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from test;", - "select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from ptest;", - "select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from test;", - "select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from ptest;" - ] - }, - { - "name": "TestDAGPlanBuilderWindow", - "cases":[ - "select lead(a, 1) over (partition by null) as c from t" - ] - }, - { - "name": "TestDAGPlanBuilderWindowParallel", - "cases":[ - "select lead(a, 1) over (partition by null) as c from t", - "select lead(a, 1) over (partition by b) as c from t" - ] - }, - { - "name": "TestNominalSort", - "cases": [ - "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by t1.a", - "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by t1.a+1", - "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by t1.a-1", - "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by -t1.a", - "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by -t1.a+3", - "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1+t1.a", - "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1-t1.a", - "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1-t1.a+3", - "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1+t1.a+3", - "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 3*t1.a" - ] - }, - { - "name": "TestInlineProjection", - "cases":[ - "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1, t2 where t1.a = t2.a;", - "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", - "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1 left outer join t2 on t1.a = t2.a;", - "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1 right outer join t2 on t1.a = t2.a;", - "select 1 from (select /*+ HASH_JOIN(t1) */ t1.a in (select t2.a from t2) from t1) x;", - "select 1 from (select /*+ HASH_JOIN(t1) */ t1.a not in (select t2.a from t2) from t1) x;", - "select /*+ INL_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", - "select /*+ INL_HASH_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", - "select /*+ INL_MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", - "select /*+ MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;" - ] - }, - { - "name": "TestHintFromDiffDatabase", - "cases": [ - "select /*+ inl_hash_join(test.t1) */ * from test.t2 join test.t1 on test.t2.a = test.t1.a" - ] ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } ] diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 675c559cf2a92..bf60c2f418f5c 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -1010,689 +1010,7 @@ "Cases": [ { "SQL": "select t1.a, (select count(t2.a) from t t2 where t2.g in (select t3.d from t t3 where t3.c = t1.a)) as agg_col from t t1;", -<<<<<<< HEAD "Best": "Apply{TableReader(Table(t))->IndexJoin{IndexReader(Index(t.c_d_e)[[NULL,+inf]]->HashAgg)->HashAgg->IndexReader(Index(t.g)[[NULL,+inf]])}(test.t3.d,test.t2.g)}->StreamAgg" -======= - "Best": "Apply{IndexReader(Index(t.f)[[NULL,+inf]])->IndexMergeJoin{IndexReader(Index(t.c_d_e)[[NULL,+inf]]->HashAgg)->HashAgg->IndexReader(Index(t.g)[[NULL,+inf]])}(test.t.d,test.t.g)}->HashAgg" - } - ] - }, - { - "Name": "TestIndexJoinHint", - "Cases": [ - { - "SQL": "select /*+ INL_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;", - "Plan": "IndexJoin{IndexLookUp(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]), Table(t1))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)" - }, - { - "SQL": "select /*+ INL_HASH_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;", - "Plan": "IndexHashJoin{IndexLookUp(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]), Table(t1))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)" - }, - { - "SQL": "select /*+ INL_MERGE_JOIN(t1) */ * from t1 join t2 on t1.a = t2.a;", - "Plan": "IndexMergeJoin{IndexLookUp(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]), Table(t1))->Projection->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)" - }, - { - "SQL": "select /*+ inl_merge_join(t2) */ t1.a, t2.a from t t1 left join t t2 use index(g_2) on t1.g=t2.g", - "Plan": "IndexMergeJoin{IndexReader(Index(t.g_3)[[NULL,+inf]])->IndexReader(Index(t.g_2)[[NULL,+inf]]->Sel([not(isnull(test.t.g))]))}(test.t.g,test.t.g)" - }, - { - "SQL": "select /*+inl_merge_join(t2)*/ t1.a, t2.a from t t1 left join t t2 use index(g_2) on t1.g=t2.g order by t1.a", - "Plan": "IndexMergeJoin{IndexReader(Index(t.g_3)[[NULL,+inf]])->IndexReader(Index(t.g_2)[[NULL,+inf]]->Sel([not(isnull(test.t.g))]))}(test.t.g,test.t.g)->Sort" - } - ] - }, - { - "Name": "TestAggToCopHint", - "Cases": [ - { - "SQL": "select /*+ AGG_TO_COP(), HASH_AGG(), USE_INDEX(t) */ sum(a) from ta group by a", - "Best": "IndexReader(Index(ta.a)[[NULL,+inf]]->HashAgg)->HashAgg", - "Warning": "[planner:1815]use_index(test.t) is inapplicable, check whether the table(test.t) exists" - }, - { - "SQL": "select /*+ AGG_TO_COP(), USE_INDEX(t) */ sum(b) from ta group by b", - "Best": "TableReader(Table(ta)->HashAgg)->HashAgg", - "Warning": "[planner:1815]use_index(test.t) is inapplicable, check whether the table(test.t) exists" - }, - { - "SQL": "select /*+ AGG_TO_COP(), HASH_AGG(), USE_INDEX(t) */ distinct a from ta group by a", - "Best": "IndexReader(Index(ta.a)[[NULL,+inf]]->HashAgg)->HashAgg", - "Warning": "[planner:1815]use_index(test.t) is inapplicable, check whether the table(test.t) exists" - }, - { - "SQL": "select /*+ AGG_TO_COP(), HASH_AGG(), HASH_JOIN(t1), USE_INDEX(t1), USE_INDEX(t2) */ sum(t1.a) from ta t1, ta t2 where t1.a = t2.b group by t1.a", - "Best": "LeftHashJoin{TableReader(Table(ta)->Sel([not(isnull(test.ta.a))]))->TableReader(Table(ta)->Sel([not(isnull(test.ta.b))]))}(test.ta.a,test.ta.b)->Projection->HashAgg", - "Warning": "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - } - ] - }, - { - "Name": "TestPushdownDistinctEnable", - "Cases": [ - { - "SQL": "select /*+ HASH_AGG() */ avg(distinct a) from t;", - "Plan": [ - "HashAgg_8 1.00 root funcs:avg(distinct Column#6)->Column#5", - "└─Projection_10 1.00 root cast(test.t.a, decimal(65,4) BINARY)->Column#6", - " └─TableReader_9 1.00 root data:HashAgg_5", - " └─HashAgg_5 1.00 cop[tikv] group by:test.t.a, ", - " └─TableFullScan_7 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Result": [ - "1.5000" - ] - }, - { - "SQL": "select /*+ HASH_AGG() */ a, count(distinct a) from t;", - "Plan": [ - "Projection_4 1.00 root test.t.a, Column#5", - "└─HashAgg_8 1.00 root funcs:count(distinct test.t.a)->Column#5, funcs:firstrow(Column#6)->test.t.a", - " └─TableReader_9 1.00 root data:HashAgg_5", - " └─HashAgg_5 1.00 cop[tikv] group by:test.t.a, funcs:firstrow(test.t.a)->Column#6", - " └─TableFullScan_7 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Result": [ - "1 2" - ] - }, - { - "SQL": "select /*+ HASH_AGG() */ avg(b), c, avg(b), count(distinct A, B), count(distinct A), count(distinct c), sum(b) from t group by c;", - "Plan": [ - "Projection_4 8000.00 root Column#5, test.t.c, Column#5, Column#6, Column#7, Column#8, Column#9", - "└─HashAgg_8 8000.00 root group by:test.t.c, funcs:avg(Column#10, Column#11)->Column#5, funcs:count(distinct test.t.a, test.t.b)->Column#6, funcs:count(distinct test.t.a)->Column#7, funcs:count(distinct test.t.c)->Column#8, funcs:sum(Column#12)->Column#9, funcs:firstrow(test.t.c)->test.t.c", - " └─TableReader_9 8000.00 root data:HashAgg_5", - " └─HashAgg_5 8000.00 cop[tikv] group by:test.t.a, test.t.b, test.t.c, funcs:count(test.t.b)->Column#10, funcs:sum(test.t.b)->Column#11, funcs:sum(test.t.b)->Column#12", - " └─TableFullScan_7 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Result": [ - "1.0000 1 1.0000 1 1 1 1", - "1.3333 3 1.3333 3 2 1 4", - "2.0000 2.0000 1 1 0 2" - ] - }, - { - "SQL": "select /*+ STREAM_AGG() */ count(distinct c) from t group by c;", - "Plan": [ - "StreamAgg_11 8000.00 root group by:test.t.c, funcs:count(distinct test.t.c)->Column#5", - "└─IndexReader_12 8000.00 root index:StreamAgg_7", - " └─StreamAgg_7 8000.00 cop[tikv] group by:test.t.c, ", - " └─IndexFullScan_10 10000.00 cop[tikv] table:t, index:c(c) keep order:true, stats:pseudo" - ], - "Result": [ - "0", - "1", - "1" - ] - }, - { - "SQL": "select /*+ STREAM_AGG() */ count(distinct c) from t;", - "Plan": [ - "StreamAgg_7 1.00 root funcs:count(distinct test.t.c)->Column#5", - "└─IndexReader_11 10000.00 root index:IndexFullScan_10", - " └─IndexFullScan_10 10000.00 cop[tikv] table:t, index:c(c) keep order:false, stats:pseudo" - ], - "Result": [ - "2" - ] - }, - { - "SQL": "select /*+ HASH_AGG() */ count(distinct c) from t;", - "Plan": [ - "HashAgg_9 1.00 root funcs:count(distinct test.t.c)->Column#5", - "└─IndexReader_10 1.00 root index:HashAgg_5", - " └─HashAgg_5 1.00 cop[tikv] group by:test.t.c, ", - " └─IndexFullScan_8 10000.00 cop[tikv] table:t, index:c(c) keep order:false, stats:pseudo" - ], - "Result": [ - "2" - ] - }, - { - "SQL": "select count(distinct c) from t group by c;", - "Plan": [ - "HashAgg_10 8000.00 root group by:test.t.c, funcs:count(distinct test.t.c)->Column#5", - "└─IndexReader_11 8000.00 root index:HashAgg_5", - " └─HashAgg_5 8000.00 cop[tikv] group by:test.t.c, ", - " └─IndexFullScan_9 10000.00 cop[tikv] table:t, index:c(c) keep order:false, stats:pseudo" - ], - "Result": [ - "0", - "1", - "1" - ] - }, - { - "SQL": "select count(distinct c) from t;", - "Plan": [ - "HashAgg_9 1.00 root funcs:count(distinct test.t.c)->Column#5", - "└─IndexReader_10 1.00 root index:HashAgg_5", - " └─HashAgg_5 1.00 cop[tikv] group by:test.t.c, ", - " └─IndexFullScan_8 10000.00 cop[tikv] table:t, index:c(c) keep order:false, stats:pseudo" - ], - "Result": [ - "2" - ] - }, - { - "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", - "Plan": [ - "HashAgg_11 1.00 root funcs:sum(distinct Column#7)->Column#4", - "└─Projection_23 16000.00 root cast(test.pt.b, decimal(65,0) BINARY)->Column#7", - " └─Union_12 16000.00 root ", - " ├─HashAgg_16 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", - " │ └─TableReader_17 8000.00 root data:HashAgg_13", - " │ └─HashAgg_13 8000.00 cop[tikv] group by:test.pt.b, ", - " │ └─TableFullScan_15 10000.00 cop[tikv] table:pt, partition:p0 keep order:false, stats:pseudo", - " └─HashAgg_21 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", - " └─TableReader_22 8000.00 root data:HashAgg_18", - " └─HashAgg_18 8000.00 cop[tikv] group by:test.pt.b, ", - " └─TableFullScan_20 10000.00 cop[tikv] table:pt, partition:p1 keep order:false, stats:pseudo" - ], - "Result": [ - "" - ] - }, - { - "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;", - "Plan": [ - "HashAgg_14 1.00 root funcs:count(distinct Column#5)->Column#6", - "└─Union_15 16000.00 root ", - " ├─HashAgg_19 8000.00 root group by:test.ta.a, funcs:firstrow(test.ta.a)->Column#5", - " │ └─TableReader_20 8000.00 root data:HashAgg_16", - " │ └─HashAgg_16 8000.00 cop[tikv] group by:test.ta.a, ", - " │ └─TableFullScan_18 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", - " └─HashAgg_24 8000.00 root group by:test.tb.a, funcs:firstrow(test.tb.a)->Column#5", - " └─TableReader_25 8000.00 root data:HashAgg_21", - " └─HashAgg_21 8000.00 cop[tikv] group by:test.tb.a, ", - " └─TableFullScan_23 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo" - ], - "Result": [ - "1" - ] - } - ] - }, - { - "Name": "TestPushdownDistinctDisable", - "Cases": [ - { - "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ avg(distinct a) from t;", - "Plan": [ - "HashAgg_5 1.00 root funcs:avg(distinct Column#6)->Column#5", - "└─Projection_8 10000.00 root cast(test.t.a, decimal(65,4) BINARY)->Column#6", - " └─TableReader_7 10000.00 root data:TableFullScan_6", - " └─TableFullScan_6 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Result": [ - "1.5000" - ] - }, - { - "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ a, count(distinct a) from t;", - "Plan": [ - "Projection_4 1.00 root test.t.a, Column#5", - "└─HashAgg_5 1.00 root funcs:count(distinct test.t.a)->Column#5, funcs:firstrow(test.t.a)->test.t.a", - " └─TableReader_7 10000.00 root data:TableFullScan_6", - " └─TableFullScan_6 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Result": [ - "1 2" - ] - }, - { - "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ avg(b), c, avg(b), count(distinct A, B), count(distinct A), count(distinct c), sum(b) from t group by c;", - "Plan": [ - "Projection_4 8000.00 root Column#5, test.t.c, Column#5, Column#6, Column#7, Column#8, Column#9", - "└─HashAgg_5 8000.00 root group by:Column#17, funcs:avg(Column#10)->Column#5, funcs:count(distinct Column#11, Column#12)->Column#6, funcs:count(distinct Column#13)->Column#7, funcs:count(distinct Column#14)->Column#8, funcs:sum(Column#15)->Column#9, funcs:firstrow(Column#16)->test.t.c", - " └─Projection_8 10000.00 root cast(test.t.b, decimal(65,4) BINARY)->Column#10, test.t.a, test.t.b, test.t.a, test.t.c, cast(test.t.b, decimal(65,0) BINARY)->Column#15, test.t.c, test.t.c", - " └─TableReader_7 10000.00 root data:TableFullScan_6", - " └─TableFullScan_6 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Result": [ - "1.0000 1 1.0000 1 1 1 1", - "1.3333 3 1.3333 3 2 1 4", - "2.0000 2.0000 1 1 0 2" - ] - }, - { - "SQL": "select /*+ STREAM_AGG(), AGG_TO_COP() */ count(distinct c) from t group by c;", - "Plan": [ - "StreamAgg_6 8000.00 root group by:test.t.c, funcs:count(distinct test.t.c)->Column#5", - "└─IndexReader_9 10000.00 root index:IndexFullScan_8", - " └─IndexFullScan_8 10000.00 cop[tikv] table:t, index:c(c) keep order:true, stats:pseudo" - ], - "Result": [ - "0", - "1", - "1" - ] - }, - { - "SQL": "select /*+ STREAM_AGG(), AGG_TO_COP() */ count(distinct c) from t;", - "Plan": [ - "StreamAgg_6 1.00 root funcs:count(distinct test.t.c)->Column#5", - "└─IndexReader_11 10000.00 root index:IndexFullScan_10", - " └─IndexFullScan_10 10000.00 cop[tikv] table:t, index:c(c) keep order:false, stats:pseudo" - ], - "Result": [ - "2" - ] - }, - { - "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct c) from t;", - "Plan": [ - "HashAgg_5 1.00 root funcs:count(distinct test.t.c)->Column#5", - "└─IndexReader_9 10000.00 root index:IndexFullScan_8", - " └─IndexFullScan_8 10000.00 cop[tikv] table:t, index:c(c) keep order:false, stats:pseudo" - ], - "Result": [ - "2" - ] - }, - { - "SQL": "select /*+ AGG_TO_COP() */ count(distinct c) from t group by c;", - "Plan": [ - "StreamAgg_6 8000.00 root group by:test.t.c, funcs:count(distinct test.t.c)->Column#5", - "└─IndexReader_12 10000.00 root index:IndexFullScan_11", - " └─IndexFullScan_11 10000.00 cop[tikv] table:t, index:c(c) keep order:true, stats:pseudo" - ], - "Result": [ - "0", - "1", - "1" - ] - }, - { - "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", - "Plan": [ - "HashAgg_11 1.00 root funcs:sum(distinct Column#7)->Column#4", - "└─Projection_23 16000.00 root cast(test.pt.b, decimal(65,0) BINARY)->Column#7", - " └─Union_12 16000.00 root ", - " ├─HashAgg_16 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", - " │ └─TableReader_17 8000.00 root data:HashAgg_13", - " │ └─HashAgg_13 8000.00 cop[tikv] group by:test.pt.b, ", - " │ └─TableFullScan_15 10000.00 cop[tikv] table:pt, partition:p0 keep order:false, stats:pseudo", - " └─HashAgg_21 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", - " └─TableReader_22 8000.00 root data:HashAgg_18", - " └─HashAgg_18 8000.00 cop[tikv] group by:test.pt.b, ", - " └─TableFullScan_20 10000.00 cop[tikv] table:pt, partition:p1 keep order:false, stats:pseudo" - ], - "Result": [ - "" - ] - }, - { - "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;", - "Plan": [ - "HashAgg_14 1.00 root funcs:count(distinct Column#5)->Column#6", - "└─Union_15 16000.00 root ", - " ├─HashAgg_19 8000.00 root group by:test.ta.a, funcs:firstrow(test.ta.a)->Column#5", - " │ └─TableReader_20 8000.00 root data:HashAgg_16", - " │ └─HashAgg_16 8000.00 cop[tikv] group by:test.ta.a, ", - " │ └─TableFullScan_18 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", - " └─HashAgg_24 8000.00 root group by:test.tb.a, funcs:firstrow(test.tb.a)->Column#5", - " └─TableReader_25 8000.00 root data:HashAgg_21", - " └─HashAgg_21 8000.00 cop[tikv] group by:test.tb.a, ", - " └─TableFullScan_23 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo" - ], - "Result": [ - "1" - ] - } - ] - }, - { - "Name": "TestPushdownDistinctEnableAggPushDownDisable", - "Cases": [ - { - "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", - "Plan": [ - "HashAgg_8 1.00 root funcs:sum(distinct Column#5)->Column#4", - "└─Projection_14 20000.00 root cast(test.pt.b, decimal(65,0) BINARY)->Column#5", - " └─Union_9 20000.00 root ", - " ├─TableReader_11 10000.00 root data:TableFullScan_10", - " │ └─TableFullScan_10 10000.00 cop[tikv] table:pt, partition:p0 keep order:false, stats:pseudo", - " └─TableReader_13 10000.00 root data:TableFullScan_12", - " └─TableFullScan_12 10000.00 cop[tikv] table:pt, partition:p1 keep order:false, stats:pseudo" - ], - "Result": [ - "" - ] - }, - { - "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;", - "Plan": [ - "HashAgg_11 1.00 root funcs:count(distinct Column#5)->Column#6", - "└─Union_12 20000.00 root ", - " ├─TableReader_15 10000.00 root data:TableFullScan_14", - " │ └─TableFullScan_14 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", - " └─TableReader_18 10000.00 root data:TableFullScan_17", - " └─TableFullScan_17 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo" - ], - "Result": [ - "1" - ] - } - ] - }, - { - "Name": "TestGroupConcatOrderby", - "Cases": [ - { - "SQL": "select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from test;", - "Plan": [ - "HashAgg_5 1.00 root funcs:group_concat(Column#6 order by Column#7 desc separator \"++\")->Column#4, funcs:group_concat(Column#8 order by Column#9 desc, Column#10 asc separator \"--\")->Column#5", - "└─Projection_18 10000.00 root cast(test.test.name, var_string(20))->Column#6, test.test.name, cast(test.test.id, var_string(20))->Column#8, test.test.name, test.test.id", - " └─TableReader_11 10000.00 root data:TableFullScan_10", - " └─TableFullScan_10 10000.00 cop[tikv] table:test keep order:false, stats:pseudo" - ], - "Result": [ - "500++200++30++20++20++10 3--3--1--1--2--1" - ] - }, - { - "SQL": "select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from ptest;", - "Plan": [ - "HashAgg_10 1.00 root funcs:group_concat(Column#6 order by Column#7 desc separator \"++\")->Column#4, funcs:group_concat(Column#8 order by Column#9 desc, Column#10 asc separator \"--\")->Column#5", - "└─Projection_23 20000.00 root cast(test.ptest.name, var_string(20))->Column#6, test.ptest.name, cast(test.ptest.id, var_string(20))->Column#8, test.ptest.name, test.ptest.id", - " └─Union_13 20000.00 root ", - " ├─TableReader_15 10000.00 root data:TableFullScan_14", - " │ └─TableFullScan_14 10000.00 cop[tikv] table:ptest, partition:p0 keep order:false, stats:pseudo", - " └─TableReader_17 10000.00 root data:TableFullScan_16", - " └─TableFullScan_16 10000.00 cop[tikv] table:ptest, partition:p1 keep order:false, stats:pseudo" - ], - "Result": [ - "500++200++30++20++20++10 3--3--1--1--2--1" - ] - }, - { - "SQL": "select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from test;", - "Plan": [ - "HashAgg_5 1.00 root funcs:group_concat(distinct Column#5 order by Column#6 desc separator \",\")->Column#4", - "└─Projection_9 10000.00 root cast(test.test.name, var_string(20))->Column#5, test.test.name", - " └─TableReader_8 10000.00 root data:TableFullScan_7", - " └─TableFullScan_7 10000.00 cop[tikv] table:test keep order:false, stats:pseudo" - ], - "Result": [ - "500,200,30,20,10" - ] - }, - { - "SQL": "select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from ptest;", - "Plan": [ - "StreamAgg_9 1.00 root funcs:group_concat(distinct Column#5 order by Column#6 desc separator \",\")->Column#4", - "└─Projection_20 20000.00 root cast(test.ptest.name, var_string(20))->Column#5, test.ptest.name", - " └─Union_15 20000.00 root ", - " ├─TableReader_17 10000.00 root data:TableFullScan_16", - " │ └─TableFullScan_16 10000.00 cop[tikv] table:ptest, partition:p0 keep order:false, stats:pseudo", - " └─TableReader_19 10000.00 root data:TableFullScan_18", - " └─TableFullScan_18 10000.00 cop[tikv] table:ptest, partition:p1 keep order:false, stats:pseudo" - ], - "Result": [ - "500,200,30,20,10" - ] - } - ] - }, - { - "Name": "TestDAGPlanBuilderWindow", - "Cases": [ - { - "SQL": "select lead(a, 1) over (partition by null) as c from t", - "Best": "IndexReader(Index(t.f)[[NULL,+inf]])->Window(lead(test.t.a, 1)->Column#14 over())->Projection" - } - ] - }, - { - "Name": "TestDAGPlanBuilderWindowParallel", - "Cases": [ - { - "SQL": "select lead(a, 1) over (partition by null) as c from t", - "Best": "IndexReader(Index(t.f)[[NULL,+inf]])->Window(lead(test.t.a, 1)->Column#14 over())->Projection" - }, - { - "SQL": "select lead(a, 1) over (partition by b) as c from t", - "Best": "TableReader(Table(t))->Sort->Window(lead(test.t.a, 1)->Column#14 over(partition by test.t.b))->Partition(execution info: concurrency:4, data source:TableReader_10)->Projection" - } - ] - }, - { - "Name": "TestNominalSort", - "Cases": [ - { - "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by t1.a", - "Plan": [ - "MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - "├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", - "│ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", - "└─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", - " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" - ], - "Result": [ - "1", - "1", - "2" - ] - }, - { - "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by t1.a+1", - "Plan": [ - "Projection_24 12487.50 root test.t.a", - "└─Projection_23 12487.50 root test.t.a, plus(test.t.a, 1)->Column#7", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", - " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", - " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", - " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" - ], - "Result": [ - "1", - "1", - "2" - ] - }, - { - "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by t1.a-1", - "Plan": [ - "Projection_24 12487.50 root test.t.a", - "└─Projection_23 12487.50 root test.t.a, minus(test.t.a, 1)->Column#7", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", - " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", - " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", - " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" - ], - "Result": [ - "1", - "1", - "2" - ] - }, - { - "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by -t1.a", - "Plan": [ - "Projection_32 12487.50 root test.t.a", - "└─Projection_31 12487.50 root test.t.a, unaryminus(test.t.a)->Column#7", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", - " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", - " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", - " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" - ], - "Result": [ - "2", - "1", - "1" - ] - }, - { - "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by -t1.a+3", - "Plan": [ - "Projection_32 12487.50 root test.t.a", - "└─Projection_31 12487.50 root test.t.a, plus(unaryminus(test.t.a), 3)->Column#7", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", - " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", - " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", - " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" - ], - "Result": [ - "2", - "1", - "1" - ] - }, - { - "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1+t1.a", - "Plan": [ - "Projection_24 12487.50 root test.t.a", - "└─Projection_23 12487.50 root test.t.a, plus(1, test.t.a)->Column#7", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", - " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", - " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", - " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" - ], - "Result": [ - "1", - "1", - "2" - ] - }, - { - "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1-t1.a", - "Plan": [ - "Projection_32 12487.50 root test.t.a", - "└─Projection_31 12487.50 root test.t.a, minus(1, test.t.a)->Column#7", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", - " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", - " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", - " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" - ], - "Result": [ - "2", - "1", - "1" - ] - }, - { - "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1-t1.a+3", - "Plan": [ - "Projection_32 12487.50 root test.t.a", - "└─Projection_31 12487.50 root test.t.a, plus(minus(1, test.t.a), 3)->Column#7", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", - " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", - " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", - " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" - ], - "Result": [ - "2", - "1", - "1" - ] - }, - { - "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 1+t1.a+3", - "Plan": [ - "Projection_24 12487.50 root test.t.a", - "└─Projection_23 12487.50 root test.t.a, plus(plus(1, test.t.a), 3)->Column#7", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", - " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", - " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", - " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" - ], - "Result": [ - "1", - "1", - "2" - ] - }, - { - "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by 3*t1.a", - "Plan": [ - "Projection_19 12487.50 root test.t.a", - "└─Sort_7 12487.50 root Column#7:asc", - " └─Projection_20 12487.50 root test.t.a, mul(3, test.t.a)->Column#7", - " └─MergeJoin_9 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_14(Build) 9990.00 root index:IndexFullScan_13", - " │ └─IndexFullScan_13 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", - " └─IndexReader_12(Probe) 9990.00 root index:IndexFullScan_11", - " └─IndexFullScan_11 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" - ], - "Result": [ - "1", - "1", - "2" - ] - } - ] - }, - { - "Name": "TestInlineProjection", - "Cases": [ - { - "SQL": "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1, t2 where t1.a = t2.a;", - "Plan": "LeftHashJoin{TableReader(Table(t1)->Sel([not(isnull(test.t1.a))]))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t1.a,test.t2.a)", - "Hints": "use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_1` `test`.`t2` ), hash_join(@`sel_1` `test`.`t1`)" - }, - { - "SQL": "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", - "Plan": "LeftHashJoin{TableReader(Table(t1)->Sel([not(isnull(test.t1.a))]))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t1.a,test.t2.a)", - "Hints": "use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_1` `test`.`t2` ), hash_join(@`sel_1` `test`.`t1`)" - }, - { - "SQL": "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1 left outer join t2 on t1.a = t2.a;", - "Plan": "LeftHashJoin{TableReader(Table(t1))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t1.a,test.t2.a)", - "Hints": "use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_1` `test`.`t2` ), hash_join(@`sel_1` `test`.`t1`)" - }, - { - "SQL": "select /*+ HASH_JOIN(t1) */ t1.b, t2.b from t1 right outer join t2 on t1.a = t2.a;", - "Plan": "RightHashJoin{TableReader(Table(t1)->Sel([not(isnull(test.t1.a))]))->TableReader(Table(t2))}(test.t1.a,test.t2.a)", - "Hints": "use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_1` `test`.`t2` ), hash_join(@`sel_1` `test`.`t1`)" - }, - { - "SQL": "select 1 from (select /*+ HASH_JOIN(t1) */ t1.a in (select t2.a from t2) from t1) x;", - "Plan": "LeftHashJoin{IndexReader(Index(t1.idx_a)[[NULL,+inf]])->IndexReader(Index(t2.idx_a)[[NULL,+inf]])}->Projection", - "Hints": "use_index(@`sel_2` `test`.`t1` `idx_a`), use_index(@`sel_3` `test`.`t2` `idx_a`), hash_join(@`sel_2` `test`.`t1`)" - }, - { - "SQL": "select 1 from (select /*+ HASH_JOIN(t1) */ t1.a not in (select t2.a from t2) from t1) x;", - "Plan": "LeftHashJoin{IndexReader(Index(t1.idx_a)[[NULL,+inf]])->IndexReader(Index(t2.idx_a)[[NULL,+inf]])}->Projection", - "Hints": "use_index(@`sel_2` `test`.`t1` `idx_a`), use_index(@`sel_3` `test`.`t2` `idx_a`), hash_join(@`sel_2` `test`.`t1`)" - }, - { - "SQL": "select /*+ INL_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", - "Plan": "IndexJoin{IndexLookUp(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]), Table(t1))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)", - "Hints": "use_index(@`sel_1` `test`.`t1` `idx_a`), use_index(@`sel_1` `test`.`t2` ), inl_join(@`sel_1` `test`.`t1`)" - }, - { - "SQL": "select /*+ INL_HASH_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", - "Plan": "IndexHashJoin{IndexLookUp(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]), Table(t1))->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)", - "Hints": "use_index(@`sel_1` `test`.`t1` `idx_a`), use_index(@`sel_1` `test`.`t2` ), inl_hash_join(@`sel_1` )" - }, - { - "SQL": "select /*+ INL_MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", - "Plan": "IndexMergeJoin{IndexLookUp(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]), Table(t1))->Projection->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)", - "Hints": "use_index(@`sel_1` `test`.`t1` `idx_a`), use_index(@`sel_1` `test`.`t2` )" - }, - { - "SQL": "select /*+ MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", - "Plan": "MergeInnerJoin{IndexLookUp(Index(t1.idx_a)[[-inf,+inf]], Table(t1))->Projection->IndexLookUp(Index(t2.idx_a)[[-inf,+inf]], Table(t2))->Projection}(test.t1.a,test.t2.a)", - "Hints": "use_index(@`sel_1` `test`.`t1` `idx_a`), use_index(@`sel_1` `test`.`t2` `idx_a`)" - } - ] - }, - { - "Name": "TestHintFromDiffDatabase", - "Cases": [ - { - "SQL": "select /*+ inl_hash_join(test.t1) */ * from test.t2 join test.t1 on test.t2.a = test.t1.a", - "Plan": "IndexHashJoin{IndexReader(Index(t2.idx_a)[[-inf,+inf]])->IndexReader(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]))}(test.t2.a,test.t1.a)" ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) } ] } From eab7282806628ecdfe250599c4e08587f186004d Mon Sep 17 00:00:00 2001 From: Feng Liyuan Date: Thu, 7 May 2020 15:37:33 +0800 Subject: [PATCH 3/9] fix conflict --- executor/aggfuncs/aggfunc_test.go | 9 ---- go.mod | 14 +++--- go.sum | 53 ++++++++++++++------ planner/core/exhaust_physical_plans.go | 3 +- planner/core/explain.go | 3 +- planner/core/rule_aggregation_push_down.go | 2 +- planner/core/rule_inject_extra_projection.go | 3 +- 7 files changed, 52 insertions(+), 35 deletions(-) diff --git a/executor/aggfuncs/aggfunc_test.go b/executor/aggfuncs/aggfunc_test.go index 487cc196d2760..72a6f9ce2132c 100644 --- a/executor/aggfuncs/aggfunc_test.go +++ b/executor/aggfuncs/aggfunc_test.go @@ -72,8 +72,6 @@ type aggTest struct { orderBy bool } -<<<<<<< HEAD -======= type multiArgsAggTest struct { dataTypes []*types.FieldType retType *types.FieldType @@ -84,7 +82,6 @@ type multiArgsAggTest struct { orderBy bool } ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) func (s *testSuite) testMergePartialResult(c *C, p aggTest) { srcChk := chunk.NewChunkWithCapacity([]*types.FieldType{p.dataType}, p.numRows) for i := 0; i < p.numRows; i++ { @@ -170,8 +167,6 @@ func buildAggTesterWithFieldType(funcName string, ft *types.FieldType, numRows i return pt } -<<<<<<< HEAD -======= func (s *testSuite) testMultiArgsMergePartialResult(c *C, p multiArgsAggTest) { srcChk := chunk.NewChunkWithCapacity(p.dataTypes, p.numRows) for i := 0; i < p.numRows; i++ { @@ -270,7 +265,6 @@ func buildMultiArgsAggTesterWithFieldType(funcName string, fts []*types.FieldTyp return mt } ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) func getDataGenFunc(ft *types.FieldType) func(i int) types.Datum { switch ft.Tp { case mysql.TypeLonglong: @@ -381,8 +375,6 @@ func (s *testSuite) testAggFunc(c *C, p aggTest) { c.Assert(err, IsNil) c.Assert(result, Equals, 0, Commentf("%v != %v", dt.String(), p.results[0])) } -<<<<<<< HEAD -======= func (s *testSuite) testMultiArgsAggFunc(c *C, p multiArgsAggTest) { srcChk := chunk.NewChunkWithCapacity(p.dataTypes, p.numRows) @@ -581,4 +573,3 @@ func (s *testSuite) baseBenchmarkAggFunc(b *testing.B, b.StartTimer() } } ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) diff --git a/go.mod b/go.mod index bf15dee371cd2..708b281f0a01f 100644 --- a/go.mod +++ b/go.mod @@ -15,7 +15,7 @@ require ( github.com/go-ole/go-ole v1.2.1 // indirect github.com/go-sql-driver/mysql v0.0.0-20170715192408-3955978caca4 github.com/gogo/protobuf v1.2.0 - github.com/golang/protobuf v1.2.0 + github.com/golang/protobuf v1.4.1 github.com/golang/snappy v0.0.1 github.com/google/btree v1.0.0 github.com/google/pprof v0.0.0-20190930153522-6ce02741cba3 @@ -38,10 +38,10 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20200311073257-e53d835099b0 github.com/pingcap/log v0.0.0-20190715063458-479153f07ebd - github.com/pingcap/parser v3.0.13-0.20200428073321-7e525711e35d+incompatible + github.com/pingcap/parser v3.0.13-0.20200507065234-2f9b5a8acf14+incompatible github.com/pingcap/pd v1.1.0-beta.0.20191223090411-ea2b748f6ee2 github.com/pingcap/tidb-tools v3.0.6-0.20191119150227-ff0a3c6e5763+incompatible - github.com/pingcap/tipb v0.0.0-20200401051346-bec3080a5428 + github.com/pingcap/tipb v0.0.0-20200426072559-d2c068e96eb3 github.com/prometheus/client_golang v0.9.0 github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910 github.com/prometheus/common v0.0.0-20181020173914-7e9e6cabbd39 // indirect @@ -50,7 +50,7 @@ require ( github.com/shirou/gopsutil v2.18.10+incompatible github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371 // indirect github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca // indirect - github.com/sirupsen/logrus v1.2.0 + github.com/sirupsen/logrus v1.6.0 github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 github.com/struCoder/pidusage v0.1.2 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 @@ -62,10 +62,10 @@ require ( go.etcd.io/etcd v0.0.0-20190320044326-77d4b742cdbf go.uber.org/atomic v1.3.2 go.uber.org/zap v1.9.1 - golang.org/x/crypto v0.0.0-20190909091759-094676da4a83 // indirect + golang.org/x/crypto v0.0.0-20200429183012-4b2356b1ed79 // indirect golang.org/x/net v0.0.0-20190909003024-a7b16738d86b - golang.org/x/sync v0.0.0-20190423024810-112230192c58 // indirect - golang.org/x/text v0.3.0 + golang.org/x/sys v0.0.0-20200501145240-bc7a7d42d5c3 // indirect + golang.org/x/text v0.3.2 golang.org/x/tools v0.0.0-20190130214255-bb1329dc71a0 google.golang.org/genproto v0.0.0-20190108161440-ae2f86662275 // indirect google.golang.org/grpc v1.17.0 diff --git a/go.sum b/go.sum index d08f490bf1c27..c3e8d23e902f5 100644 --- a/go.sum +++ b/go.sum @@ -68,6 +68,13 @@ github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfb github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0 h1:P3YflyNX/ehuJFLhxviNdFxQPkGK5cDcApsge1SqnvM= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.1 h1:ZFgWrT+bLgsYPirOnRfKLYJLvssAegOj/hgyMFdJZe0= +github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= @@ -75,6 +82,10 @@ github.com/google/btree v0.0.0-20180124185431-e89373fe6b4a/go.mod h1:lNA+9X1NB3Z github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/pprof v0.0.0-20190930153522-6ce02741cba3 h1:3CYI9xg87xNAD+es02gZxbX/ky4KQeoFBsNOzuoAQZg= github.com/google/pprof v0.0.0-20190930153522-6ce02741cba3/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/shlex v0.0.0-20181106134648-c34317bd91bf/go.mod h1:RpwtwJQFrIEPstU94h88MWPXP2ektJZ8cZ0YntAmXiE= @@ -108,8 +119,8 @@ github.com/juju/ratelimit v1.0.1/go.mod h1:qapgC/Gy+xNh9UxzV13HGGl/6UXNN+ct+vwSg github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 h1:2U0HzY8BJ8hVwDKIzp7y4voR9CX/nvcfymLmg2UiOio= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= -github.com/konsorten/go-windows-terminal-sequences v1.0.1 h1:mweAR1A6xJ3oS2pRaGiHgQ4OO8tzTaLawm8vnODuwDk= -github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= +github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pty v1.0.0/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= @@ -157,14 +168,16 @@ github.com/pingcap/kvproto v0.0.0-20200311073257-e53d835099b0 h1:dXXNHvDwAEN1YNg github.com/pingcap/kvproto v0.0.0-20200311073257-e53d835099b0/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY= github.com/pingcap/log v0.0.0-20190715063458-479153f07ebd h1:hWDol43WY5PGhsh3+8794bFHY1bPrmu6bTalpssCrGg= github.com/pingcap/log v0.0.0-20190715063458-479153f07ebd/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v3.0.13-0.20200428073321-7e525711e35d+incompatible h1:t/vYVkoW99IdMdDKCL1SbvHRvsbcg5fwTatehu2iDoU= -github.com/pingcap/parser v3.0.13-0.20200428073321-7e525711e35d+incompatible/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v3.0.13-0.20200507065234-2f9b5a8acf14+incompatible h1:Z/L+KFD/8adlmXix/7AcgidRAaf9TNwcglcX1TcqLzs= +github.com/pingcap/parser v3.0.13-0.20200507065234-2f9b5a8acf14+incompatible/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v1.1.0-beta.0.20191223090411-ea2b748f6ee2 h1:NL23b8tsg6M1QpSQedK14/Jx++QeyKL2rGiBvXAQVfA= github.com/pingcap/pd v1.1.0-beta.0.20191223090411-ea2b748f6ee2/go.mod h1:b4gaAPSxaVVtaB+EHamV4Nsv8JmTdjlw0cTKmp4+dRQ= github.com/pingcap/tidb-tools v3.0.6-0.20191119150227-ff0a3c6e5763+incompatible h1:I8HirWsu1MZp6t9G/g8yKCEjJJxtHooKakEgccvdJ4M= github.com/pingcap/tidb-tools v3.0.6-0.20191119150227-ff0a3c6e5763+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20200401051346-bec3080a5428 h1:u2eGnp74AlgviPKRcf49MESRp8RFAhURomJsL2XYC6o= -github.com/pingcap/tipb v0.0.0-20200401051346-bec3080a5428/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= +github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee h1:XJQ6/LGzOSc/jo33AD8t7jtc4GohxcyODsYnb+kZXJM= +github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= +github.com/pingcap/tipb v0.0.0-20200426072559-d2c068e96eb3 h1:rwil8cOXl/TYlhAhvu8LG05Ukw6eRnCRmEbR0uyodbc= +github.com/pingcap/tipb v0.0.0-20200426072559-d2c068e96eb3/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -194,8 +207,8 @@ github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371/go.mod h1:ZY1cvUeJ github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca h1:3fECS8atRjByijiI8yYiuwLwQ2ZxXobW7ua/8GRB3pI= github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/sirupsen/logrus v1.0.5/go.mod h1:pMByvHTf9Beacp5x1UXfOR9xyW/9antXMhjMPG0dEzc= -github.com/sirupsen/logrus v1.2.0 h1:juTguoYk5qI21pwyTXY3B3Y5cOTH3ZUyZCg1v/mihuo= -github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= +github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= +github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ= @@ -204,7 +217,6 @@ github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3 github.com/spf13/pflag v1.0.1 h1:aCvUg6QPl3ibpQUxyLkrEkCHtPqYJL4x9AuhqVqFis4= github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0 h1:TivCn/peBQ7UY8ooIcPgZFpTNSz0Q2U6UrFlUfqbe0Q= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= @@ -247,10 +259,9 @@ go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/ go.uber.org/zap v1.9.1 h1:XCJQEf3W6eZaVwhRBof6ImoYGJSITeKWsyeh3HFu/5o= go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= golang.org/x/crypto v0.0.0-20180608092829-8ac0e0d97ce4/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20190909091759-094676da4a83 h1:mgAKeshyNqWKdENOnQsg+8dRTwZFIwFaO3HNl52sweA= -golang.org/x/crypto v0.0.0-20190909091759-094676da4a83/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200429183012-4b2356b1ed79 h1:IaQbIIB2X/Mp/DKctl6ROxz1KyMlKp4uyvL6+kQ7C88= +golang.org/x/crypto v0.0.0-20200429183012-4b2356b1ed79/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -263,21 +274,26 @@ golang.org/x/net v0.0.0-20190909003024-a7b16738d86b/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190423024810-112230192c58 h1:8gQV6CLnAEikrhgkHFbMAEhagSSnXWGV915qUMm9mrU= -golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d h1:+R4KGOnez64A81RvjARKc4UT5/tI9ujCIVX+P5KiHuI= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200501145240-bc7a7d42d5c3 h1:5B6i6EAiSYyejWfvc5Rc9BbI3rzIsrrXfAQBWnYfn+w= +golang.org/x/sys v0.0.0-20200501145240-bc7a7d42d5c3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2 h1:+DCIGbF/swA92ohVg0//6X2IVY3KZs6p9mix0ziNYJM= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190130214255-bb1329dc71a0 h1:iRpjPej1fPzmfoBhMFkp3HdqzF+ytPmAwiQhJGV0zGw= golang.org/x/tools v0.0.0-20190130214255-bb1329dc71a0/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/genproto v0.0.0-20180608181217-32ee49c4dd80/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= @@ -289,6 +305,13 @@ google.golang.org/grpc v1.14.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmE google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio= google.golang.org/grpc v1.17.0 h1:TRJYBgMclJvGYn2rIMjj+h9KtMt5r1Ij7ODVRIZkwhk= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.22.0 h1:cJv5/xdbk1NnMPR1VP9+HU6gupuG9MLBoH1r6RHZ2MY= +google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= gopkg.in/airbrake/gobrake.v2 v2.0.9/go.mod h1:/h5ZAUhDkGaJfjzjKLSjv6zCL6O0LLBxU4K+aSYdM/U= gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20180810215634-df19058c872c/go.mod h1:3HH7i1SgMqlzxCcBmUHW657sD4Kvv9sC3HpL3YukzwA= diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 9e719ca437685..3510e9fc7c106 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" @@ -1121,7 +1122,7 @@ func (lt *LogicalTopN) getPhysLimits() []PhysicalPlan { } // Check if this prop's columns can match by items totally. -func matchItems(p *property.PhysicalProperty, items []*ByItems) bool { +func matchItems(p *property.PhysicalProperty, items []*util.ByItems) bool { if len(items) < len(p.Items) { return false } diff --git a/planner/core/explain.go b/planner/core/explain.go index b4461bdef0a19..09ed11f522188 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/statistics" ) @@ -459,7 +460,7 @@ func (p *PhysicalWindow) formatFrameBound(buffer *bytes.Buffer, bound *FrameBoun } } -func explainNormalizedByItems(buffer *bytes.Buffer, byItems []*ByItems) *bytes.Buffer { +func explainNormalizedByItems(buffer *bytes.Buffer, byItems []*util.ByItems) *bytes.Buffer { for i, item := range byItems { order := "asc" if item.Desc { diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index d3de39c5f8729..8c31cf555d3e8 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -57,7 +57,7 @@ func (a *aggregationPushDownSolver) isDecomposableWithUnion(fun *aggregation.Agg return false } switch fun.Name { - case ast.AggFuncGroupConcat, ast.AggFuncVarPop, ast.AggFuncJsonObjectAgg: + case ast.AggFuncGroupConcat, ast.AggFuncVarPop: return false case ast.AggFuncMax, ast.AggFuncMin, ast.AggFuncFirstRow: return true diff --git a/planner/core/rule_inject_extra_projection.go b/planner/core/rule_inject_extra_projection.go index 89fe281080104..f4533c412a69d 100644 --- a/planner/core/rule_inject_extra_projection.go +++ b/planner/core/rule_inject_extra_projection.go @@ -119,8 +119,9 @@ func injectProjBelowAgg(aggPlan PhysicalPlan, aggFuncs []*aggregation.AggFuncDes } projExprs = append(projExprs, byItem.Expr) newArg := &expression.Column{ - UniqueID: aggPlan.SCtx().GetSessionVars().AllocPlanColumnID(), + UniqueID: aggPlan.context().GetSessionVars().AllocPlanColumnID(), RetType: byItem.Expr.GetType(), + ColName: model.NewCIStr(fmt.Sprintf("col_%d", len(projSchemaCols))), Index: cursor, } projSchemaCols = append(projSchemaCols, newArg) From a78dfe11ee2eb9fccea943be60edc1c3ad2b851f Mon Sep 17 00:00:00 2001 From: Feng Liyuan Date: Thu, 7 May 2020 16:15:07 +0800 Subject: [PATCH 4/9] save --- go.mod | 10 +- go.sum | 43 +- .../transformation_rules_suite_out.json | 2356 ----------------- 3 files changed, 16 insertions(+), 2393 deletions(-) delete mode 100644 planner/cascades/testdata/transformation_rules_suite_out.json diff --git a/go.mod b/go.mod index 708b281f0a01f..3136480dae580 100644 --- a/go.mod +++ b/go.mod @@ -15,7 +15,7 @@ require ( github.com/go-ole/go-ole v1.2.1 // indirect github.com/go-sql-driver/mysql v0.0.0-20170715192408-3955978caca4 github.com/gogo/protobuf v1.2.0 - github.com/golang/protobuf v1.4.1 + github.com/golang/protobuf v1.2.0 github.com/golang/snappy v0.0.1 github.com/google/btree v1.0.0 github.com/google/pprof v0.0.0-20190930153522-6ce02741cba3 @@ -50,7 +50,7 @@ require ( github.com/shirou/gopsutil v2.18.10+incompatible github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371 // indirect github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca // indirect - github.com/sirupsen/logrus v1.6.0 + github.com/sirupsen/logrus v1.2.0 github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 github.com/struCoder/pidusage v0.1.2 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 @@ -62,10 +62,10 @@ require ( go.etcd.io/etcd v0.0.0-20190320044326-77d4b742cdbf go.uber.org/atomic v1.3.2 go.uber.org/zap v1.9.1 - golang.org/x/crypto v0.0.0-20200429183012-4b2356b1ed79 // indirect + golang.org/x/crypto v0.0.0-20190909091759-094676da4a83 // indirect golang.org/x/net v0.0.0-20190909003024-a7b16738d86b - golang.org/x/sys v0.0.0-20200501145240-bc7a7d42d5c3 // indirect - golang.org/x/text v0.3.2 + golang.org/x/sync v0.0.0-20190423024810-112230192c58 // indirect + golang.org/x/text v0.3.0 golang.org/x/tools v0.0.0-20190130214255-bb1329dc71a0 google.golang.org/genproto v0.0.0-20190108161440-ae2f86662275 // indirect google.golang.org/grpc v1.17.0 diff --git a/go.sum b/go.sum index ad8b35d348876..ca0a52d9bb703 100644 --- a/go.sum +++ b/go.sum @@ -68,13 +68,6 @@ github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfb github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0 h1:P3YflyNX/ehuJFLhxviNdFxQPkGK5cDcApsge1SqnvM= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= -github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= -github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= -github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= -github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= -github.com/golang/protobuf v1.4.1 h1:ZFgWrT+bLgsYPirOnRfKLYJLvssAegOj/hgyMFdJZe0= -github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= @@ -82,10 +75,6 @@ github.com/google/btree v0.0.0-20180124185431-e89373fe6b4a/go.mod h1:lNA+9X1NB3Z github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4= -github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/pprof v0.0.0-20190930153522-6ce02741cba3 h1:3CYI9xg87xNAD+es02gZxbX/ky4KQeoFBsNOzuoAQZg= github.com/google/pprof v0.0.0-20190930153522-6ce02741cba3/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/shlex v0.0.0-20181106134648-c34317bd91bf/go.mod h1:RpwtwJQFrIEPstU94h88MWPXP2ektJZ8cZ0YntAmXiE= @@ -119,8 +108,8 @@ github.com/juju/ratelimit v1.0.1/go.mod h1:qapgC/Gy+xNh9UxzV13HGGl/6UXNN+ct+vwSg github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 h1:2U0HzY8BJ8hVwDKIzp7y4voR9CX/nvcfymLmg2UiOio= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= -github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= -github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/konsorten/go-windows-terminal-sequences v1.0.1 h1:mweAR1A6xJ3oS2pRaGiHgQ4OO8tzTaLawm8vnODuwDk= +github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pty v1.0.0/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= @@ -205,8 +194,8 @@ github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371/go.mod h1:ZY1cvUeJ github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca h1:3fECS8atRjByijiI8yYiuwLwQ2ZxXobW7ua/8GRB3pI= github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/sirupsen/logrus v1.0.5/go.mod h1:pMByvHTf9Beacp5x1UXfOR9xyW/9antXMhjMPG0dEzc= -github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= -github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= +github.com/sirupsen/logrus v1.2.0 h1:juTguoYk5qI21pwyTXY3B3Y5cOTH3ZUyZCg1v/mihuo= +github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ= @@ -215,6 +204,7 @@ github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3 github.com/spf13/pflag v1.0.1 h1:aCvUg6QPl3ibpQUxyLkrEkCHtPqYJL4x9AuhqVqFis4= github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0 h1:TivCn/peBQ7UY8ooIcPgZFpTNSz0Q2U6UrFlUfqbe0Q= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= @@ -257,9 +247,10 @@ go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/ go.uber.org/zap v1.9.1 h1:XCJQEf3W6eZaVwhRBof6ImoYGJSITeKWsyeh3HFu/5o= go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= golang.org/x/crypto v0.0.0-20180608092829-8ac0e0d97ce4/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20200429183012-4b2356b1ed79 h1:IaQbIIB2X/Mp/DKctl6ROxz1KyMlKp4uyvL6+kQ7C88= -golang.org/x/crypto v0.0.0-20200429183012-4b2356b1ed79/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20190909091759-094676da4a83 h1:mgAKeshyNqWKdENOnQsg+8dRTwZFIwFaO3HNl52sweA= +golang.org/x/crypto v0.0.0-20190909091759-094676da4a83/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -272,26 +263,21 @@ golang.org/x/net v0.0.0-20190909003024-a7b16738d86b/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58 h1:8gQV6CLnAEikrhgkHFbMAEhagSSnXWGV915qUMm9mrU= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d h1:+R4KGOnez64A81RvjARKc4UT5/tI9ujCIVX+P5KiHuI= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200501145240-bc7a7d42d5c3 h1:5B6i6EAiSYyejWfvc5Rc9BbI3rzIsrrXfAQBWnYfn+w= -golang.org/x/sys v0.0.0-20200501145240-bc7a7d42d5c3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= -golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2 h1:+DCIGbF/swA92ohVg0//6X2IVY3KZs6p9mix0ziNYJM= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190130214255-bb1329dc71a0 h1:iRpjPej1fPzmfoBhMFkp3HdqzF+ytPmAwiQhJGV0zGw= golang.org/x/tools v0.0.0-20190130214255-bb1329dc71a0/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= -golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/genproto v0.0.0-20180608181217-32ee49c4dd80/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= @@ -303,13 +289,6 @@ google.golang.org/grpc v1.14.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmE google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio= google.golang.org/grpc v1.17.0 h1:TRJYBgMclJvGYn2rIMjj+h9KtMt5r1Ij7ODVRIZkwhk= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= -google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= -google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= -google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= -google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= -google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= -google.golang.org/protobuf v1.22.0 h1:cJv5/xdbk1NnMPR1VP9+HU6gupuG9MLBoH1r6RHZ2MY= -google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= gopkg.in/airbrake/gobrake.v2 v2.0.9/go.mod h1:/h5ZAUhDkGaJfjzjKLSjv6zCL6O0LLBxU4K+aSYdM/U= gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20180810215634-df19058c872c/go.mod h1:3HH7i1SgMqlzxCcBmUHW657sD4Kvv9sC3HpL3YukzwA= diff --git a/planner/cascades/testdata/transformation_rules_suite_out.json b/planner/cascades/testdata/transformation_rules_suite_out.json deleted file mode 100644 index ba8bbcae8e221..0000000000000 --- a/planner/cascades/testdata/transformation_rules_suite_out.json +++ /dev/null @@ -1,2356 +0,0 @@ -[ - { - "Name": "TestPredicatePushDown", - "Cases": [ - { - "SQL": "select a, b from (select a, b from t as t1 order by a) as t2 where t2.b > 10", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b]", - " Projection_5 input:[Group#1], test.t.a, test.t.b", - "Group#1 Schema:[test.t.a,test.t.b]", - " Sort_3 input:[Group#2], test.t.a:asc", - "Group#2 Schema:[test.t.a,test.t.b]", - " Projection_2 input:[Group#3], test.t.a, test.t.b", - "Group#3 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_8 input:[Group#4], table:t1", - "Group#4 Schema:[test.t.a,test.t.b]", - " Selection_9 input:[Group#5], gt(test.t.b, 10)", - "Group#5 Schema:[test.t.a,test.t.b]", - " TableScan_7 table:t1, pk col:test.t.a" - ] - }, - { - "SQL": "select a, b from (select a, b from t as t1 order by a) as t2 where t2.a > 10", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b]", - " Projection_5 input:[Group#1], test.t.a, test.t.b", - "Group#1 Schema:[test.t.a,test.t.b]", - " Sort_3 input:[Group#2], test.t.a:asc", - "Group#2 Schema:[test.t.a,test.t.b]", - " Projection_2 input:[Group#3], test.t.a, test.t.b", - "Group#3 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_8 input:[Group#4], table:t1", - "Group#4 Schema:[test.t.a,test.t.b]", - " TableScan_10 table:t1, pk col:test.t.a, cond:[gt(test.t.a, 10)]" - ] - }, - { - "SQL": "select a, b from (select a, b, a+b as a_b from t as t1) as t2 where a_b > 10 and b = 1", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b]", - " Projection_4 input:[Group#1], test.t.a, test.t.b", - "Group#1 Schema:[test.t.a,test.t.b,Column#13]", - " Projection_2 input:[Group#2], test.t.a, test.t.b, plus(test.t.a, test.t.b)->Column#13", - "Group#2 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_7 input:[Group#3], table:t1", - "Group#3 Schema:[test.t.a,test.t.b]", - " Selection_8 input:[Group#4], eq(test.t.b, 1), gt(plus(test.t.a, test.t.b), 10)", - "Group#4 Schema:[test.t.a,test.t.b]", - " TableScan_6 table:t1, pk col:test.t.a" - ] - }, - { - "SQL": "select b, @i:=@i+1 as ii from (select b, @i:=0 from t as t1) as t2 where @i < 10", - "Result": [ - "Group#0 Schema:[test.t.b,Column#14]", - " Projection_4 input:[Group#1], test.t.b, setvar(i, cast(plus(cast(getvar(i), double BINARY), 1), var_string(5)))->Column#14", - "Group#1 Schema:[test.t.b,Column#13]", - " Selection_3 input:[Group#2], lt(cast(getvar(\"i\")), 10)", - "Group#2 Schema:[test.t.b,Column#13]", - " Projection_2 input:[Group#3], test.t.b, setvar(i, 0)->Column#13", - "Group#3 Schema:[test.t.b]", - " TiKVSingleGather_6 input:[Group#4], table:t1", - "Group#4 Schema:[test.t.b]", - " TableScan_5 table:t1" - ] - }, - { - "SQL": "select b, @i:=@i+1 as ii from (select a, b, @i:=0 from t as t1) as t2 where @i < 10 and a > 10", - "Result": [ - "Group#0 Schema:[test.t.b,Column#14]", - " Projection_4 input:[Group#1], test.t.b, setvar(i, cast(plus(cast(getvar(i), double BINARY), 1), var_string(5)))->Column#14", - "Group#1 Schema:[test.t.a,test.t.b,Column#13]", - " Selection_6 input:[Group#2], lt(cast(getvar(\"i\")), 10)", - "Group#2 Schema:[test.t.a,test.t.b,Column#13]", - " Projection_2 input:[Group#3], test.t.a, test.t.b, setvar(i, 0)->Column#13", - "Group#3 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_8 input:[Group#4], table:t1", - "Group#4 Schema:[test.t.a,test.t.b]", - " TableScan_10 table:t1, pk col:test.t.a, cond:[gt(test.t.a, 10)]" - ] - }, - { - "SQL": "select a, max(b) from t group by a having a > 1", - "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], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", - "Group#2 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_8 input:[Group#3], table:t", - "Group#3 Schema:[test.t.a,test.t.b]", - " TableScan_10 table:t, pk col:test.t.a, cond:[gt(test.t.a, 1)]" - ] - }, - { - "SQL": "select a, avg(b) from t group by a having a > 1 and max(b) > 10", - "Result": [ - "Group#0 Schema:[test.t.a,Column#16]", - " Projection_5 input:[Group#1], test.t.a, Column#13", - "Group#1 Schema:[test.t.a,Column#13,Column#14]", - " Projection_3 input:[Group#2], test.t.a, Column#13, Column#14", - "Group#2 Schema:[Column#13,Column#14,test.t.a]", - " Selection_8 input:[Group#3], gt(Column#14, 10)", - "Group#3 Schema:[Column#13,Column#14,test.t.a]", - " Aggregation_2 input:[Group#4], group by:test.t.a, funcs:avg(test.t.b), max(test.t.b), firstrow(test.t.a)", - "Group#4 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_10 input:[Group#5], table:t", - "Group#5 Schema:[test.t.a,test.t.b]", - " TableScan_12 table:t, pk col:test.t.a, cond:[gt(test.t.a, 1)]" - ] - }, - { - "SQL": "select t1.a, t1.b, t2.b from t t1, t t2 where t1.a = t2.a and t2.b = t1.b and t1.a > 10 and t2.b > 10 and t1.a > t2.b", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.b]", - " Projection_5 input:[Group#1], test.t.a, test.t.b, test.t.b", - "Group#1 Schema:[test.t.a,test.t.b,test.t.a,test.t.b]", - " Join_3 input:[Group#2,Group#3], inner join, equal:[eq(test.t.a, test.t.a) eq(test.t.b, test.t.b)], other cond:gt(test.t.a, test.t.b)", - "Group#2 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_9 input:[Group#4], table:t1", - "Group#4 Schema:[test.t.a,test.t.b]", - " Selection_12 input:[Group#5], gt(test.t.a, test.t.b), gt(test.t.b, 10)", - "Group#5 Schema:[test.t.a,test.t.b]", - " TableScan_11 table:t1, pk col:test.t.a, cond:[gt(test.t.a, 10)]", - "Group#3 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_14 input:[Group#6], table:t2", - "Group#6 Schema:[test.t.a,test.t.b]", - " Selection_17 input:[Group#7], gt(test.t.a, test.t.b), gt(test.t.b, 10)", - "Group#7 Schema:[test.t.a,test.t.b]", - " TableScan_16 table:t2, pk col:test.t.a, cond:[gt(test.t.a, 10)]" - ] - }, - { - "SQL": "select t1.a, t1.b from t t1, t t2 where t1.a = t2.a and t1.a = 10 and t2.a = 5", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b]", - " Projection_5 input:[Group#1], test.t.a, test.t.b", - "Group#1 Schema:[test.t.a,test.t.b,test.t.a]", - " TableDual_6 rowcount:0" - ] - }, - { - "SQL": "select a, f from t where f > 1", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.f]", - " Projection_3 input:[Group#1], test.t.a, test.t.f", - "Group#1 Schema:[test.t.a,test.t.f]", - " TiKVSingleGather_5 input:[Group#2], table:t", - " TiKVSingleGather_7 input:[Group#3], table:t, index:f", - " TiKVSingleGather_9 input:[Group#4], table:t, index:f_g", - "Group#2 Schema:[test.t.a,test.t.f]", - " Selection_10 input:[Group#5], gt(test.t.f, 1)", - "Group#5 Schema:[test.t.a,test.t.f]", - " TableScan_4 table:t, pk col:test.t.a", - "Group#3 Schema:[test.t.a,test.t.f]", - " IndexScan_13 table:t, index:f, cond:[gt(test.t.f, 1)]", - "Group#4 Schema:[test.t.a,test.t.f]", - " IndexScan_14 table:t, index:f, g, cond:[gt(test.t.f, 1)]" - ] - }, - { - "SQL": "select a, f from (select a, f, g from t where f = 1) t1 where g > 1", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.f]", - " Projection_5 input:[Group#1], test.t.a, test.t.f", - "Group#1 Schema:[test.t.a,test.t.f,test.t.g]", - " Projection_3 input:[Group#2], test.t.a, test.t.f, test.t.g", - "Group#2 Schema:[test.t.a,test.t.f,test.t.g]", - " TiKVSingleGather_9 input:[Group#3], table:t", - " TiKVSingleGather_11 input:[Group#4], table:t, index:f_g", - "Group#3 Schema:[test.t.a,test.t.f,test.t.g]", - " Selection_12 input:[Group#5], eq(test.t.f, 1), gt(test.t.g, 1)", - "Group#5 Schema:[test.t.a,test.t.f,test.t.g]", - " TableScan_8 table:t, pk col:test.t.a", - "Group#4 Schema:[test.t.a,test.t.f,test.t.g]", - " IndexScan_14 table:t, index:f, g, cond:[eq(test.t.f, 1) gt(test.t.g, 1)]" - ] - }, - { - "SQL": "select a, f from t where g > 1 and f > 1", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.f]", - " Projection_3 input:[Group#1], test.t.a, test.t.f", - "Group#1 Schema:[test.t.a,test.t.f,test.t.g]", - " TiKVSingleGather_5 input:[Group#2], table:t", - " TiKVSingleGather_7 input:[Group#3], table:t, index:f_g", - "Group#2 Schema:[test.t.a,test.t.f,test.t.g]", - " Selection_8 input:[Group#4], gt(test.t.f, 1), gt(test.t.g, 1)", - "Group#4 Schema:[test.t.a,test.t.f,test.t.g]", - " TableScan_4 table:t, pk col:test.t.a", - "Group#3 Schema:[test.t.a,test.t.f,test.t.g]", - " Selection_11 input:[Group#5], gt(test.t.g, 1)", - "Group#5 Schema:[test.t.a,test.t.f,test.t.g]", - " IndexScan_10 table:t, index:f, g, cond:[gt(test.t.f, 1)]" - ] - }, - { - "SQL": "select t1.a, t1.b from t t1, t t2 where t1.a = t2.a and t1.a = 10 and t2.a = 5", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b]", - " Projection_5 input:[Group#1], test.t.a, test.t.b", - "Group#1 Schema:[test.t.a,test.t.b,test.t.a]", - " TableDual_6 rowcount:0" - ] - }, - { - "SQL": "select a, b from ((select a, b from t) union all(select c as a, d as b from t)) as t1 where a > 1", - "Result": [ - "Group#0 Schema:[Column#25,Column#26]", - " Projection_9 input:[Group#1], Column#25, Column#26", - "Group#1 Schema:[Column#25,Column#26]", - " Union_5 input:[Group#2,Group#3]", - "Group#2 Schema:[Column#25,Column#26]", - " Projection_6 input:[Group#4], test.t.a, test.t.b", - "Group#4 Schema:[test.t.a,test.t.b]", - " Projection_4 input:[Group#5], test.t.a, test.t.b", - "Group#5 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_15 input:[Group#6], table:t", - "Group#6 Schema:[test.t.a,test.t.b]", - " TableScan_17 table:t, pk col:test.t.a, cond:[gt(test.t.a, 1)]", - "Group#3 Schema:[Column#25,Column#26]", - " Projection_7 input:[Group#7], test.t.c, test.t.d", - "Group#7 Schema:[test.t.c,test.t.d]", - " Projection_2 input:[Group#8], test.t.c, test.t.d", - "Group#8 Schema:[test.t.c,test.t.d]", - " TiKVSingleGather_19 input:[Group#9], table:t", - " TiKVSingleGather_21 input:[Group#10], table:t, index:c_d_e", - "Group#9 Schema:[test.t.c,test.t.d]", - " Selection_22 input:[Group#11], gt(test.t.c, 1)", - "Group#11 Schema:[test.t.c,test.t.d]", - " TableScan_18 table:t", - "Group#10 Schema:[test.t.c,test.t.d]", - " IndexScan_24 table:t, index:c, d, e, cond:[gt(test.t.c, 1)]" - ] - }, - { - "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", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b]", - " Projection_7 input:[Group#1], test.t.a, test.t.b", - "Group#1 Schema:[test.t.a,test.t.b,Column#14]", - " Projection_5 input:[Group#2], test.t.a, test.t.b, Column#14", - "Group#2 Schema:[test.t.a,test.t.b,Column#14]", - " Selection_10 input:[Group#3], eq(test.t.b, Column#14), lt(test.t.a, 10)", - "Group#3 Schema:[test.t.a,test.t.b,Column#14]", - " Window_4 input:[Group#4]", - "Group#4 Schema:[test.t.a,test.t.b]", - " Projection_3 input:[Group#5], test.t.a, test.t.b", - "Group#5 Schema:[test.t.a,test.t.b]", - " Projection_2 input:[Group#6], test.t.a, test.t.b", - "Group#6 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_14 input:[Group#7], table:t", - "Group#7 Schema:[test.t.a,test.t.b]", - " Selection_15 input:[Group#8], gt(test.t.b, 10)", - "Group#8 Schema:[test.t.a,test.t.b]", - " TableScan_13 table:t, pk col:test.t.a" - ] - }, - { - "SQL": "select b, c from (select b, c from t where b > 1 and c > 1) as t1 where b > 2 and c > 2", - "Result": [ - "Group#0 Schema:[test.t.b,test.t.c]", - " Projection_5 input:[Group#1], test.t.b, test.t.c", - "Group#1 Schema:[test.t.b,test.t.c]", - " Projection_3 input:[Group#2], test.t.b, test.t.c", - "Group#2 Schema:[test.t.b,test.t.c]", - " TiKVSingleGather_9 input:[Group#3], table:t", - "Group#3 Schema:[test.t.b,test.t.c]", - " Selection_10 input:[Group#4], gt(test.t.b, 1), gt(test.t.b, 2), gt(test.t.c, 1), gt(test.t.c, 2)", - "Group#4 Schema:[test.t.b,test.t.c]", - " TableScan_8 table:t" - ] - } - ] - }, - { - "Name": "TestAggPushDownGather", - "Cases": [ - { - "SQL": "select b, sum(a) from t group by b", - "Result": [ - "Group#0 Schema:[test.t.b,Column#13], UniqueKey:[test.t.b]", - " Projection_3 input:[Group#1], test.t.b, Column#13", - "Group#1 Schema:[Column#13,test.t.b], UniqueKey:[test.t.b]", - " Aggregation_2 input:[Group#2], group by:test.t.b, funcs:sum(test.t.a), firstrow(test.t.b)", - " Aggregation_7 input:[Group#3], group by:test.t.b, funcs:sum(Column#14), firstrow(test.t.b)", - "Group#2 Schema:[test.t.a,test.t.b], UniqueKey:[test.t.a]", - " TiKVSingleGather_5 input:[Group#4], table:t", - "Group#4 Schema:[test.t.a,test.t.b], UniqueKey:[test.t.a]", - " TableScan_4 table:t, pk col:test.t.a", - "Group#3 Schema:[Column#14,test.t.b]", - " TiKVSingleGather_5 input:[Group#5], table:t", - "Group#5 Schema:[Column#14,test.t.b]", - " Aggregation_6 input:[Group#4], group by:test.t.b, funcs:sum(test.t.a)" - ] - }, - { - "SQL": "select b, sum(a) from t group by c, b", - "Result": [ - "Group#0 Schema:[test.t.b,Column#13]", - " Projection_3 input:[Group#1], test.t.b, Column#13", - "Group#1 Schema:[Column#13,test.t.b]", - " Aggregation_2 input:[Group#2], group by:test.t.b, test.t.c, funcs:sum(test.t.a), firstrow(test.t.b)", - " Aggregation_7 input:[Group#3], group by:test.t.b, test.t.c, funcs:sum(Column#14), firstrow(test.t.b)", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c], UniqueKey:[test.t.a]", - " TiKVSingleGather_5 input:[Group#4], table:t", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c], UniqueKey:[test.t.a]", - " TableScan_4 table:t, pk col:test.t.a", - "Group#3 Schema:[Column#14,test.t.c,test.t.b]", - " TiKVSingleGather_5 input:[Group#5], table:t", - "Group#5 Schema:[Column#14,test.t.c,test.t.b]", - " Aggregation_6 input:[Group#4], group by:test.t.b, test.t.c, funcs:sum(test.t.a)" - ] - }, - { - "SQL": "select b, sum(a) from t group by sin(b)+sin(c), b", - "Result": [ - "Group#0 Schema:[test.t.b,Column#13]", - " Projection_3 input:[Group#1], test.t.b, Column#13", - "Group#1 Schema:[Column#13,test.t.b]", - " Aggregation_2 input:[Group#2], group by:plus(sin(cast(test.t.b)), sin(cast(test.t.c))), test.t.b, funcs:sum(test.t.a), firstrow(test.t.b)", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c], UniqueKey:[test.t.a]", - " TiKVSingleGather_5 input:[Group#3], table:t", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c], UniqueKey:[test.t.a]", - " TableScan_4 table:t, pk col:test.t.a" - ] - } - ] - }, - { - "Name": "TestTopNRules", - "Cases": [ - { - "SQL": "select b from t order by a limit 2", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_5 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b,test.t.a]", - " Projection_2 input:[Group#2], test.t.b, test.t.a", - "Group#2 Schema:[test.t.a,test.t.b]", - " TopN_7 input:[Group#3], test.t.a:asc, offset:0, count:2", - "Group#3 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_9 input:[Group#4], table:t", - "Group#4 Schema:[test.t.a,test.t.b]", - " TopN_10 input:[Group#5], test.t.a:asc, offset:0, count:2", - "Group#5 Schema:[test.t.a,test.t.b]", - " TableScan_8 table:t, pk col:test.t.a" - ] - }, - { - "SQL": "select b from t limit 2", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_2 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b]", - " Limit_3 input:[Group#2], offset:0, count:2", - "Group#2 Schema:[test.t.b]", - " TiKVSingleGather_5 input:[Group#3], table:t", - "Group#3 Schema:[test.t.b]", - " Limit_6 input:[Group#4], offset:0, count:2", - "Group#4 Schema:[test.t.b]", - " TableScan_4 table:t" - ] - }, - { - "SQL": "select a+b from t order by a limit 1 offset 2", - "Result": [ - "Group#0 Schema:[Column#14]", - " Projection_5 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13,test.t.a]", - " Projection_2 input:[Group#2], plus(test.t.a, test.t.b)->Column#13, test.t.a", - "Group#2 Schema:[test.t.a,test.t.b]", - " TopN_7 input:[Group#3], test.t.a:asc, offset:2, count:1", - "Group#3 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_9 input:[Group#4], table:t", - "Group#4 Schema:[test.t.a,test.t.b]", - " TopN_10 input:[Group#5], test.t.a:asc, offset:0, count:3", - "Group#5 Schema:[test.t.a,test.t.b]", - " TableScan_8 table:t, pk col:test.t.a" - ] - }, - { - "SQL": "select c from t order by t.a limit 1", - "Result": [ - "Group#0 Schema:[test.t.c]", - " Projection_5 input:[Group#1], test.t.c", - "Group#1 Schema:[test.t.c,test.t.a]", - " Projection_2 input:[Group#2], test.t.c, test.t.a", - "Group#2 Schema:[test.t.a,test.t.c]", - " TopN_7 input:[Group#3], test.t.a:asc, offset:0, count:1", - " TopN_7 input:[Group#4], test.t.a:asc, offset:0, count:1", - "Group#3 Schema:[test.t.a,test.t.c]", - " TiKVSingleGather_11 input:[Group#5], table:t, index:c_d_e", - "Group#5 Schema:[test.t.a,test.t.c]", - " TopN_13 input:[Group#6], test.t.a:asc, offset:0, count:1", - "Group#6 Schema:[test.t.a,test.t.c]", - " IndexScan_10 table:t, index:c, d, e", - "Group#4 Schema:[test.t.a,test.t.c]", - " TiKVSingleGather_9 input:[Group#7], table:t", - "Group#7 Schema:[test.t.a,test.t.c]", - " TopN_12 input:[Group#8], test.t.a:asc, offset:0, count:1", - "Group#8 Schema:[test.t.a,test.t.c]", - " TableScan_8 table:t, pk col:test.t.a" - ] - }, - { - "SQL": "select c from t order by t.a + t.b limit 1", - "Result": [ - "Group#0 Schema:[test.t.c]", - " Projection_5 input:[Group#1], test.t.c", - "Group#1 Schema:[test.t.c,test.t.a,test.t.b]", - " Projection_2 input:[Group#2], test.t.c, test.t.a, test.t.b", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", - " TopN_7 input:[Group#3], plus(test.t.a, test.t.b):asc, offset:0, count:1", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c]", - " TiKVSingleGather_9 input:[Group#4], table:t", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c]", - " TopN_10 input:[Group#5], plus(test.t.a, test.t.b):asc, offset:0, count:1", - "Group#5 Schema:[test.t.a,test.t.b,test.t.c]", - " TableScan_8 table:t, pk col:test.t.a" - ] - }, - { - "SQL": "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)", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c]", - " Projection_9 input:[Group#1], test.t.a, test.t.b, test.t.c", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c]", - " Apply_8 input:[Group#2,Group#3], semi join, equal:[eq(test.t.a, test.t.a)]", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", - " TiKVSingleGather_13 input:[Group#4], table:t1", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c]", - " TableScan_12 table:t1, pk col:test.t.a", - "Group#3 Schema:[test.t.a]", - " Projection_5 input:[Group#5], test.t.a", - "Group#5 Schema:[test.t.a,test.t.b]", - " TopN_11 input:[Group#6], , offset:0, count:1", - "Group#6 Schema:[test.t.a,test.t.b]", - " Selection_4 input:[Group#7], gt(test.t.b, test.t.b)", - "Group#7 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_15 input:[Group#8], table:t2", - "Group#8 Schema:[test.t.a,test.t.b]", - " TableScan_14 table:t2, pk col:test.t.a" - ] - }, - { - "SQL": "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)", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c]", - " Projection_11 input:[Group#1], test.t.a, test.t.b, test.t.c", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c]", - " Apply_10 input:[Group#2,Group#3], semi join, equal:[eq(test.t.a, test.t.a)]", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", - " TiKVSingleGather_16 input:[Group#4], table:t1", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c]", - " TableScan_15 table:t1, pk col:test.t.a", - "Group#3 Schema:[test.t.a]", - " Projection_9 input:[Group#5], test.t.a", - "Group#5 Schema:[test.t.a,Column#25]", - " Projection_6 input:[Group#6], test.t.a, Column#25", - "Group#6 Schema:[test.t.a,Column#25]", - " Projection_5 input:[Group#7], test.t.a, test.t.b", - "Group#7 Schema:[test.t.a,test.t.b]", - " TopN_14 input:[Group#8], test.t.b:asc, offset:0, count:1", - "Group#8 Schema:[test.t.a,test.t.b]", - " Selection_4 input:[Group#9], gt(test.t.b, test.t.b)", - "Group#9 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_18 input:[Group#10], table:t2", - "Group#10 Schema:[test.t.a,test.t.b]", - " TableScan_17 table:t2, pk col:test.t.a" - ] - }, - { - "SQL": "select a, b from (select @i as a, @i := @i+1 as b from t) t order by a desc limit 1", - "Result": [ - "Group#0 Schema:[Column#13,Column#14]", - " Projection_3 input:[Group#1], Column#13, Column#14", - "Group#1 Schema:[Column#13,Column#14]", - " TopN_7 input:[Group#2], Column#13:desc, offset:0, count:1", - "Group#2 Schema:[Column#13,Column#14]", - " Projection_2 input:[Group#3], getvar(i)->Column#13, setvar(i, cast(plus(cast(getvar(i), double BINARY), 1), var_string(5)))->Column#14", - "Group#3 Schema:[test.t.a]", - " TiKVSingleGather_9 input:[Group#4], table:t", - " TiKVSingleGather_21 input:[Group#5], table:t, index:e_d_c_str_prefix", - " TiKVSingleGather_19 input:[Group#6], table:t, index:c_d_e_str", - " TiKVSingleGather_17 input:[Group#7], table:t, index:f_g", - " TiKVSingleGather_15 input:[Group#8], table:t, index:g", - " TiKVSingleGather_13 input:[Group#9], table:t, index:f", - " TiKVSingleGather_11 input:[Group#10], table:t, index:c_d_e", - "Group#4 Schema:[test.t.a]", - " TableScan_8 table:t, pk col:test.t.a", - "Group#5 Schema:[test.t.a]", - " IndexScan_20 table:t, index:e_str, d_str, c_str", - "Group#6 Schema:[test.t.a]", - " IndexScan_18 table:t, index:c_str, d_str, e_str", - "Group#7 Schema:[test.t.a]", - " IndexScan_16 table:t, index:f, g", - "Group#8 Schema:[test.t.a]", - " IndexScan_14 table:t, index:g", - "Group#9 Schema:[test.t.a]", - " IndexScan_12 table:t, index:f", - "Group#10 Schema:[test.t.a]", - " IndexScan_10 table:t, index:c, d, e" - ] - }, - { - "SQL": "select * from t t1 left join t t2 on t1.b = t2.b order by t1.b limit 1", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TopN_8 input:[Group#2], test.t.b:asc, offset:0, count:1", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Join_3 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.b, test.t.b)]", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TopN_9 input:[Group#5], test.t.b:asc, offset:0, count:1", - "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_11 input:[Group#6], table:t1", - "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TopN_12 input:[Group#7], test.t.b:asc, offset:0, count:1", - "Group#7 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_10 table:t1, pk col:test.t.a", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_14 input:[Group#8], table:t2", - "Group#8 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_13 table:t2, pk col:test.t.a" - ] - }, - { - "SQL": "select * from t t1 left join t t2 on t1.b = t2.b order by t1.a, t1.c limit 1", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TopN_8 input:[Group#2], test.t.a:asc, test.t.c:asc, offset:0, count:1", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Join_3 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.b, test.t.b)]", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TopN_9 input:[Group#5], test.t.a:asc, test.t.c:asc, offset:0, count:1", - "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_11 input:[Group#6], table:t1", - "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TopN_12 input:[Group#7], test.t.a:asc, test.t.c:asc, offset:0, count:1", - "Group#7 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_10 table:t1, pk col:test.t.a", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_14 input:[Group#8], table:t2", - "Group#8 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_13 table:t2, pk col:test.t.a" - ] - }, - { - "SQL": "select * from t t1 left join t t2 on t1.b = t2.b order by t2.a, t2.c limit 1", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TopN_8 input:[Group#2], test.t.a:asc, test.t.c:asc, offset:0, count:1", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Join_3 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.b, test.t.b)]", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_10 input:[Group#5], table:t1", - "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_9 table:t1, pk col:test.t.a", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_12 input:[Group#6], table:t2", - "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_11 table:t2, pk col:test.t.a" - ] - }, - { - "SQL": "select * from t t1 left join t t2 on t1.b = t2.b order by t1.a, t2.c limit 1", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TopN_8 input:[Group#2], test.t.a:asc, test.t.c:asc, offset:0, count:1", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Join_3 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.b, test.t.b)]", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_10 input:[Group#5], table:t1", - "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_9 table:t1, pk col:test.t.a", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_12 input:[Group#6], table:t2", - "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_11 table:t2, pk col:test.t.a" - ] - }, - { - "SQL": "select * from t t1 right join t t2 on t1.b = t2.b order by t1.a, t1.c limit 1", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TopN_8 input:[Group#2], test.t.a:asc, test.t.c:asc, offset:0, count:1", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Join_3 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.b, test.t.b)]", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_10 input:[Group#5], table:t1", - "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_9 table:t1, pk col:test.t.a", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_12 input:[Group#6], table:t2", - "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_11 table:t2, pk col:test.t.a" - ] - }, - { - "SQL": "select * from t t1 right join t t2 on t1.b = t2.b order by t2.a, t2.c limit 1", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TopN_8 input:[Group#2], test.t.a:asc, test.t.c:asc, offset:0, count:1", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Join_3 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.b, test.t.b)]", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_11 input:[Group#5], table:t1", - "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_10 table:t1, pk col:test.t.a", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TopN_9 input:[Group#6], test.t.a:asc, test.t.c:asc, offset:0, count:1", - "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_13 input:[Group#7], table:t2", - "Group#7 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TopN_14 input:[Group#8], test.t.a:asc, test.t.c:asc, offset:0, count:1", - "Group#8 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_12 table:t2, pk col:test.t.a" - ] - }, - { - "SQL": "select * from t t1 right join t t2 on t1.b = t2.b order by t1.a, t2.c limit 1", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TopN_8 input:[Group#2], test.t.a:asc, test.t.c:asc, offset:0, count:1", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Join_3 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.b, test.t.b)]", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_10 input:[Group#5], table:t1", - "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_9 table:t1, pk col:test.t.a", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_12 input:[Group#6], table:t2", - "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_11 table:t2, pk col:test.t.a" - ] - }, - { - "SQL": "select * from t t1 left join t t2 on t1.b = t2.b limit 1", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_5 input:[Group#2], offset:0, count:1", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Join_3 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.b, test.t.b)]", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_6 input:[Group#5], offset:0, count:1", - "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_8 input:[Group#6], table:t1", - "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_9 input:[Group#7], offset:0, count:1", - "Group#7 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_7 table:t1, pk col:test.t.a", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_11 input:[Group#8], table:t2", - "Group#8 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_10 table:t2, pk col:test.t.a" - ] - }, - { - "SQL": "select * from t t1 left join t t2 on t1.b = t2.b limit 5 offset 4", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_5 input:[Group#2], offset:4, count:5", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Join_3 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.b, test.t.b)]", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_6 input:[Group#5], offset:0, count:9", - "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_8 input:[Group#6], table:t1", - "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_9 input:[Group#7], offset:0, count:9", - "Group#7 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_7 table:t1, pk col:test.t.a", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_11 input:[Group#8], table:t2", - "Group#8 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_10 table:t2, pk col:test.t.a" - ] - }, - { - "SQL": "select * from t t1 right join t t2 on t1.b = t2.b limit 1", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_5 input:[Group#2], offset:0, count:1", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Join_3 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.b, test.t.b)]", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_8 input:[Group#5], table:t1", - "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_7 table:t1, pk col:test.t.a", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_6 input:[Group#6], offset:0, count:1", - "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_10 input:[Group#7], table:t2", - "Group#7 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_11 input:[Group#8], offset:0, count:1", - "Group#8 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_9 table:t2, pk col:test.t.a" - ] - }, - { - "SQL": "select * from t t1 right join t t2 on t1.b = t2.b limit 5 offset 4", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Projection_4 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_5 input:[Group#2], offset:4, count:5", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Join_3 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.b, test.t.b)]", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_8 input:[Group#5], table:t1", - "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_7 table:t1, pk col:test.t.a", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_6 input:[Group#6], offset:0, count:9", - "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_10 input:[Group#7], table:t2", - "Group#7 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_11 input:[Group#8], offset:0, count:9", - "Group#8 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_9 table:t2, pk col:test.t.a" - ] - }, - { - "SQL": "(select a from t) union all (select b from t) order by a limit 2;", - "Result": [ - "Group#0 Schema:[Column#25]", - " TopN_10 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_13 input:[Group#6], test.t.a:asc, offset:0, count:2", - " TopN_13 input:[Group#7], test.t.a:asc, offset:0, count:2", - " TopN_13 input:[Group#8], test.t.a:asc, offset:0, count:2", - " TopN_13 input:[Group#9], test.t.a:asc, offset:0, count:2", - " TopN_13 input:[Group#10], test.t.a:asc, offset:0, count:2", - " TopN_13 input:[Group#11], test.t.a:asc, offset:0, count:2", - " TopN_13 input:[Group#12], test.t.a:asc, offset:0, count:2", - "Group#6 Schema:[test.t.a]", - " TiKVSingleGather_19 input:[Group#13], table:t, index:c_d_e", - "Group#13 Schema:[test.t.a]", - " TopN_36 input:[Group#14], test.t.a:asc, offset:0, count:2", - "Group#14 Schema:[test.t.a]", - " IndexScan_18 table:t, index:c, d, e", - "Group#7 Schema:[test.t.a]", - " TiKVSingleGather_21 input:[Group#15], table:t, index:f", - "Group#15 Schema:[test.t.a]", - " TopN_35 input:[Group#16], test.t.a:asc, offset:0, count:2", - "Group#16 Schema:[test.t.a]", - " IndexScan_20 table:t, index:f", - "Group#8 Schema:[test.t.a]", - " TiKVSingleGather_23 input:[Group#17], table:t, index:g", - "Group#17 Schema:[test.t.a]", - " TopN_34 input:[Group#18], test.t.a:asc, offset:0, count:2", - "Group#18 Schema:[test.t.a]", - " IndexScan_22 table:t, index:g", - "Group#9 Schema:[test.t.a]", - " TiKVSingleGather_25 input:[Group#19], table:t, index:f_g", - "Group#19 Schema:[test.t.a]", - " TopN_33 input:[Group#20], test.t.a:asc, offset:0, count:2", - "Group#20 Schema:[test.t.a]", - " IndexScan_24 table:t, index:f, g", - "Group#10 Schema:[test.t.a]", - " TiKVSingleGather_27 input:[Group#21], table:t, index:c_d_e_str", - "Group#21 Schema:[test.t.a]", - " TopN_32 input:[Group#22], test.t.a:asc, offset:0, count:2", - "Group#22 Schema:[test.t.a]", - " IndexScan_26 table:t, index:c_str, d_str, e_str", - "Group#11 Schema:[test.t.a]", - " TiKVSingleGather_29 input:[Group#23], table:t, index:e_d_c_str_prefix", - "Group#23 Schema:[test.t.a]", - " TopN_31 input:[Group#24], test.t.a:asc, offset:0, count:2", - "Group#24 Schema:[test.t.a]", - " IndexScan_28 table:t, index:e_str, d_str, c_str", - "Group#12 Schema:[test.t.a]", - " TiKVSingleGather_17 input:[Group#25], table:t", - "Group#25 Schema:[test.t.a]", - " TopN_30 input:[Group#26], test.t.a:asc, offset:0, count:2", - "Group#26 Schema:[test.t.a]", - " TableScan_16 table:t, pk col:test.t.a", - "Group#3 Schema:[Column#25]", - " Projection_7 input:[Group#27], test.t.b", - "Group#27 Schema:[test.t.b]", - " Projection_2 input:[Group#28], test.t.b", - "Group#28 Schema:[test.t.b]", - " TopN_15 input:[Group#29], test.t.b:asc, offset:0, count:2", - "Group#29 Schema:[test.t.b]", - " TiKVSingleGather_38 input:[Group#30], table:t", - "Group#30 Schema:[test.t.b]", - " TopN_39 input:[Group#31], test.t.b:asc, offset:0, count:2", - "Group#31 Schema:[test.t.b]", - " TableScan_37 table:t" - ] - }, - { - "SQL": "(select a from t) union all (select b from t) limit 2;", - "Result": [ - "Group#0 Schema:[Column#25]", - " Limit_8 input:[Group#1], 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]", - " Limit_9 input:[Group#6], offset:0, count:2", - " Limit_9 input:[Group#7], offset:0, count:2", - " Limit_9 input:[Group#8], offset:0, count:2", - " Limit_9 input:[Group#9], offset:0, count:2", - " Limit_9 input:[Group#10], offset:0, count:2", - " Limit_9 input:[Group#11], offset:0, count:2", - " Limit_9 input:[Group#12], offset:0, count:2", - "Group#6 Schema:[test.t.a]", - " TiKVSingleGather_13 input:[Group#13], table:t, index:c_d_e", - "Group#13 Schema:[test.t.a]", - " Limit_30 input:[Group#14], offset:0, count:2", - "Group#14 Schema:[test.t.a]", - " IndexScan_12 table:t, index:c, d, e", - "Group#7 Schema:[test.t.a]", - " TiKVSingleGather_15 input:[Group#15], table:t, index:f", - "Group#15 Schema:[test.t.a]", - " Limit_29 input:[Group#16], offset:0, count:2", - "Group#16 Schema:[test.t.a]", - " IndexScan_14 table:t, index:f", - "Group#8 Schema:[test.t.a]", - " TiKVSingleGather_17 input:[Group#17], table:t, index:g", - "Group#17 Schema:[test.t.a]", - " Limit_28 input:[Group#18], offset:0, count:2", - "Group#18 Schema:[test.t.a]", - " IndexScan_16 table:t, index:g", - "Group#9 Schema:[test.t.a]", - " TiKVSingleGather_19 input:[Group#19], table:t, index:f_g", - "Group#19 Schema:[test.t.a]", - " Limit_27 input:[Group#20], offset:0, count:2", - "Group#20 Schema:[test.t.a]", - " IndexScan_18 table:t, index:f, g", - "Group#10 Schema:[test.t.a]", - " TiKVSingleGather_21 input:[Group#21], table:t, index:c_d_e_str", - "Group#21 Schema:[test.t.a]", - " Limit_26 input:[Group#22], offset:0, count:2", - "Group#22 Schema:[test.t.a]", - " IndexScan_20 table:t, index:c_str, d_str, e_str", - "Group#11 Schema:[test.t.a]", - " TiKVSingleGather_23 input:[Group#23], table:t, index:e_d_c_str_prefix", - "Group#23 Schema:[test.t.a]", - " Limit_25 input:[Group#24], offset:0, count:2", - "Group#24 Schema:[test.t.a]", - " IndexScan_22 table:t, index:e_str, d_str, c_str", - "Group#12 Schema:[test.t.a]", - " TiKVSingleGather_11 input:[Group#25], table:t", - "Group#25 Schema:[test.t.a]", - " Limit_24 input:[Group#26], offset:0, count:2", - "Group#26 Schema:[test.t.a]", - " TableScan_10 table:t, pk col:test.t.a", - "Group#3 Schema:[Column#25]", - " Projection_7 input:[Group#27], test.t.b", - "Group#27 Schema:[test.t.b]", - " Projection_2 input:[Group#28], test.t.b", - "Group#28 Schema:[test.t.b]", - " Limit_9 input:[Group#29], offset:0, count:2", - "Group#29 Schema:[test.t.b]", - " TiKVSingleGather_32 input:[Group#30], table:t", - "Group#30 Schema:[test.t.b]", - " Limit_33 input:[Group#31], offset:0, count:2", - "Group#31 Schema:[test.t.b]", - " TableScan_31 table:t" - ] - }, - { - "SQL": "(select a from t) union all (select b from t) limit 2 offset 5;", - "Result": [ - "Group#0 Schema:[Column#25]", - " Limit_8 input:[Group#1], offset:5, 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]", - " Limit_9 input:[Group#6], offset:0, count:7", - " Limit_9 input:[Group#7], offset:0, count:7", - " Limit_9 input:[Group#8], offset:0, count:7", - " Limit_9 input:[Group#9], offset:0, count:7", - " Limit_9 input:[Group#10], offset:0, count:7", - " Limit_9 input:[Group#11], offset:0, count:7", - " Limit_9 input:[Group#12], offset:0, count:7", - "Group#6 Schema:[test.t.a]", - " TiKVSingleGather_13 input:[Group#13], table:t, index:c_d_e", - "Group#13 Schema:[test.t.a]", - " Limit_30 input:[Group#14], offset:0, count:7", - "Group#14 Schema:[test.t.a]", - " IndexScan_12 table:t, index:c, d, e", - "Group#7 Schema:[test.t.a]", - " TiKVSingleGather_15 input:[Group#15], table:t, index:f", - "Group#15 Schema:[test.t.a]", - " Limit_29 input:[Group#16], offset:0, count:7", - "Group#16 Schema:[test.t.a]", - " IndexScan_14 table:t, index:f", - "Group#8 Schema:[test.t.a]", - " TiKVSingleGather_17 input:[Group#17], table:t, index:g", - "Group#17 Schema:[test.t.a]", - " Limit_28 input:[Group#18], offset:0, count:7", - "Group#18 Schema:[test.t.a]", - " IndexScan_16 table:t, index:g", - "Group#9 Schema:[test.t.a]", - " TiKVSingleGather_19 input:[Group#19], table:t, index:f_g", - "Group#19 Schema:[test.t.a]", - " Limit_27 input:[Group#20], offset:0, count:7", - "Group#20 Schema:[test.t.a]", - " IndexScan_18 table:t, index:f, g", - "Group#10 Schema:[test.t.a]", - " TiKVSingleGather_21 input:[Group#21], table:t, index:c_d_e_str", - "Group#21 Schema:[test.t.a]", - " Limit_26 input:[Group#22], offset:0, count:7", - "Group#22 Schema:[test.t.a]", - " IndexScan_20 table:t, index:c_str, d_str, e_str", - "Group#11 Schema:[test.t.a]", - " TiKVSingleGather_23 input:[Group#23], table:t, index:e_d_c_str_prefix", - "Group#23 Schema:[test.t.a]", - " Limit_25 input:[Group#24], offset:0, count:7", - "Group#24 Schema:[test.t.a]", - " IndexScan_22 table:t, index:e_str, d_str, c_str", - "Group#12 Schema:[test.t.a]", - " TiKVSingleGather_11 input:[Group#25], table:t", - "Group#25 Schema:[test.t.a]", - " Limit_24 input:[Group#26], offset:0, count:7", - "Group#26 Schema:[test.t.a]", - " TableScan_10 table:t, pk col:test.t.a", - "Group#3 Schema:[Column#25]", - " Projection_7 input:[Group#27], test.t.b", - "Group#27 Schema:[test.t.b]", - " Projection_2 input:[Group#28], test.t.b", - "Group#28 Schema:[test.t.b]", - " Limit_9 input:[Group#29], offset:0, count:7", - "Group#29 Schema:[test.t.b]", - " TiKVSingleGather_32 input:[Group#30], table:t", - "Group#30 Schema:[test.t.b]", - " Limit_33 input:[Group#31], offset:0, count:7", - "Group#31 Schema:[test.t.b]", - " TableScan_31 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_12 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, decimal(65,0) BINARY)->Column#26", - "Group#4 Schema:[test.t.a]", - " Projection_6 input:[Group#5], test.t.a", - "Group#5 Schema:[test.t.a]", - " TopN_15 input:[Group#6], cast(test.t.a):asc, offset:0, count:2", - " TopN_15 input:[Group#7], cast(test.t.a):asc, offset:0, count:2", - " TopN_15 input:[Group#8], cast(test.t.a):asc, offset:0, count:2", - " TopN_15 input:[Group#9], cast(test.t.a):asc, offset:0, count:2", - " TopN_15 input:[Group#10], cast(test.t.a):asc, offset:0, count:2", - " TopN_15 input:[Group#11], cast(test.t.a):asc, offset:0, count:2", - " TopN_15 input:[Group#12], cast(test.t.a):asc, offset:0, count:2", - "Group#6 Schema:[test.t.a]", - " TiKVSingleGather_21 input:[Group#13], table:t, index:c_d_e", - "Group#13 Schema:[test.t.a]", - " TopN_38 input:[Group#14], cast(test.t.a):asc, offset:0, count:2", - "Group#14 Schema:[test.t.a]", - " IndexScan_20 table:t, index:c, d, e", - "Group#7 Schema:[test.t.a]", - " TiKVSingleGather_23 input:[Group#15], table:t, index:f", - "Group#15 Schema:[test.t.a]", - " TopN_37 input:[Group#16], cast(test.t.a):asc, offset:0, count:2", - "Group#16 Schema:[test.t.a]", - " IndexScan_22 table:t, index:f", - "Group#8 Schema:[test.t.a]", - " TiKVSingleGather_25 input:[Group#17], table:t, index:g", - "Group#17 Schema:[test.t.a]", - " TopN_36 input:[Group#18], cast(test.t.a):asc, offset:0, count:2", - "Group#18 Schema:[test.t.a]", - " IndexScan_24 table:t, index:g", - "Group#9 Schema:[test.t.a]", - " TiKVSingleGather_27 input:[Group#19], table:t, index:f_g", - "Group#19 Schema:[test.t.a]", - " TopN_35 input:[Group#20], cast(test.t.a):asc, offset:0, count:2", - "Group#20 Schema:[test.t.a]", - " IndexScan_26 table:t, index:f, g", - "Group#10 Schema:[test.t.a]", - " TiKVSingleGather_29 input:[Group#21], table:t, index:c_d_e_str", - "Group#21 Schema:[test.t.a]", - " TopN_34 input:[Group#22], cast(test.t.a):asc, offset:0, count:2", - "Group#22 Schema:[test.t.a]", - " IndexScan_28 table:t, index:c_str, d_str, e_str", - "Group#11 Schema:[test.t.a]", - " TiKVSingleGather_31 input:[Group#23], table:t, index:e_d_c_str_prefix", - "Group#23 Schema:[test.t.a]", - " TopN_33 input:[Group#24], cast(test.t.a):asc, offset:0, count:2", - "Group#24 Schema:[test.t.a]", - " IndexScan_30 table:t, index:e_str, d_str, c_str", - "Group#12 Schema:[test.t.a]", - " TiKVSingleGather_19 input:[Group#25], table:t", - "Group#25 Schema:[test.t.a]", - " TopN_32 input:[Group#26], cast(test.t.a):asc, offset:0, count:2", - "Group#26 Schema:[test.t.a]", - " TableScan_18 table:t, pk col:test.t.a", - "Group#3 Schema:[Column#26]", - " Projection_9 input:[Group#27], cast(Column#13, decimal(65,0) BINARY)->Column#26", - "Group#27 Schema:[Column#13]", - " Projection_4 input:[Group#28], Column#13", - "Group#28 Schema:[Column#13]", - " TopN_17 input:[Group#29], cast(Column#13):asc, offset:0, count:2", - "Group#29 Schema:[Column#13]", - " Aggregation_3 input:[Group#30], group by:test.t.b, funcs:sum(test.t.a)", - "Group#30 Schema:[test.t.a,test.t.b]", - " Selection_2 input:[Group#31], gt(test.t.a, 2)", - "Group#31 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_40 input:[Group#32], table:t", - "Group#32 Schema:[test.t.a,test.t.b]", - " TableScan_39 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_12 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, decimal(65,0) BINARY)->Column#26", - "Group#4 Schema:[test.t.a]", - " Projection_6 input:[Group#5], test.t.a", - "Group#5 Schema:[test.t.a]", - " TopN_15 input:[Group#6], cast(test.t.a):asc, offset:0, count:3", - " TopN_15 input:[Group#7], cast(test.t.a):asc, offset:0, count:3", - " TopN_15 input:[Group#8], cast(test.t.a):asc, offset:0, count:3", - " TopN_15 input:[Group#9], cast(test.t.a):asc, offset:0, count:3", - " TopN_15 input:[Group#10], cast(test.t.a):asc, offset:0, count:3", - " TopN_15 input:[Group#11], cast(test.t.a):asc, offset:0, count:3", - " TopN_15 input:[Group#12], cast(test.t.a):asc, offset:0, count:3", - "Group#6 Schema:[test.t.a]", - " TiKVSingleGather_21 input:[Group#13], table:t, index:c_d_e", - "Group#13 Schema:[test.t.a]", - " TopN_38 input:[Group#14], cast(test.t.a):asc, offset:0, count:3", - "Group#14 Schema:[test.t.a]", - " IndexScan_20 table:t, index:c, d, e", - "Group#7 Schema:[test.t.a]", - " TiKVSingleGather_23 input:[Group#15], table:t, index:f", - "Group#15 Schema:[test.t.a]", - " TopN_37 input:[Group#16], cast(test.t.a):asc, offset:0, count:3", - "Group#16 Schema:[test.t.a]", - " IndexScan_22 table:t, index:f", - "Group#8 Schema:[test.t.a]", - " TiKVSingleGather_25 input:[Group#17], table:t, index:g", - "Group#17 Schema:[test.t.a]", - " TopN_36 input:[Group#18], cast(test.t.a):asc, offset:0, count:3", - "Group#18 Schema:[test.t.a]", - " IndexScan_24 table:t, index:g", - "Group#9 Schema:[test.t.a]", - " TiKVSingleGather_27 input:[Group#19], table:t, index:f_g", - "Group#19 Schema:[test.t.a]", - " TopN_35 input:[Group#20], cast(test.t.a):asc, offset:0, count:3", - "Group#20 Schema:[test.t.a]", - " IndexScan_26 table:t, index:f, g", - "Group#10 Schema:[test.t.a]", - " TiKVSingleGather_29 input:[Group#21], table:t, index:c_d_e_str", - "Group#21 Schema:[test.t.a]", - " TopN_34 input:[Group#22], cast(test.t.a):asc, offset:0, count:3", - "Group#22 Schema:[test.t.a]", - " IndexScan_28 table:t, index:c_str, d_str, e_str", - "Group#11 Schema:[test.t.a]", - " TiKVSingleGather_31 input:[Group#23], table:t, index:e_d_c_str_prefix", - "Group#23 Schema:[test.t.a]", - " TopN_33 input:[Group#24], cast(test.t.a):asc, offset:0, count:3", - "Group#24 Schema:[test.t.a]", - " IndexScan_30 table:t, index:e_str, d_str, c_str", - "Group#12 Schema:[test.t.a]", - " TiKVSingleGather_19 input:[Group#25], table:t", - "Group#25 Schema:[test.t.a]", - " TopN_32 input:[Group#26], cast(test.t.a):asc, offset:0, count:3", - "Group#26 Schema:[test.t.a]", - " TableScan_18 table:t, pk col:test.t.a", - "Group#3 Schema:[Column#26]", - " Projection_9 input:[Group#27], cast(Column#13, decimal(65,0) BINARY)->Column#26", - "Group#27 Schema:[Column#13]", - " Projection_4 input:[Group#28], Column#13", - "Group#28 Schema:[Column#13]", - " TopN_17 input:[Group#29], cast(Column#13):asc, offset:0, count:3", - "Group#29 Schema:[Column#13]", - " Aggregation_3 input:[Group#30], group by:test.t.b, funcs:sum(test.t.a)", - "Group#30 Schema:[test.t.a,test.t.b]", - " Selection_2 input:[Group#31], gt(test.t.a, 2)", - "Group#31 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_40 input:[Group#32], table:t", - "Group#32 Schema:[test.t.a,test.t.b]", - " TableScan_39 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_16 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_19 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_25 input:[Group#9], table:t", - " TiKVSingleGather_37 input:[Group#10], table:t, index:e_d_c_str_prefix", - " TiKVSingleGather_35 input:[Group#11], table:t, index:c_d_e_str", - " TiKVSingleGather_33 input:[Group#12], table:t, index:f_g", - " TiKVSingleGather_31 input:[Group#13], table:t, index:g", - " TiKVSingleGather_29 input:[Group#14], table:t, index:f", - " TiKVSingleGather_27 input:[Group#15], table:t, index:c_d_e", - "Group#9 Schema:[test.t.a]", - " TableScan_24 table:t, pk col:test.t.a", - "Group#10 Schema:[test.t.a]", - " IndexScan_36 table:t, index:e_str, d_str, c_str", - "Group#11 Schema:[test.t.a]", - " IndexScan_34 table:t, index:c_str, d_str, e_str", - "Group#12 Schema:[test.t.a]", - " IndexScan_32 table:t, index:f, g", - "Group#13 Schema:[test.t.a]", - " IndexScan_30 table:t, index:g", - "Group#14 Schema:[test.t.a]", - " IndexScan_28 table:t, index:f", - "Group#15 Schema:[test.t.a]", - " IndexScan_26 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_21 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_39 input:[Group#20], table:t", - "Group#20 Schema:[test.t.a,test.t.b]", - " TableScan_38 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_23 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_41 input:[Group#25], table:t", - " TiKVSingleGather_43 input:[Group#26], table:t, index:c_d_e", - "Group#25 Schema:[test.t.a,test.t.c]", - " TableScan_40 table:t, pk col:test.t.a", - "Group#26 Schema:[test.t.a,test.t.c]", - " IndexScan_42 table:t, index:c, d, e" - ] - } - ] - }, - { - "Name": "TestProjectionElimination", - "Cases": [ - { - "SQL": "select a, b from (select a, b from t) as t2", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select a+b from (select a, b from t) as t2", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], plus(test.t.a, test.t.b)->Column#13", - "Group#1 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select a from (select floor(a) as a from t) as t2", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_2 input:[Group#1], floor(test.t.a)->Column#13", - "Group#1 Schema:[test.t.a]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select a from (select a, b from (select a, b, c from t) as t2) as t3", - "Result": [ - "Group#0 Schema:[test.t.a]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select a+c from (select floor(a) as a, b, c from t) as t2", - "Result": [ - "Group#0 Schema:[Column#14]", - " Projection_4 input:[Group#1], plus(floor(test.t.a), test.t.c)->Column#14", - "Group#1 Schema:[test.t.a,test.t.c]", - " DataSource_1 table:t" - ] - } - ] - }, - { - "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]", - " DataSource_1 table:t", - "Group#3 Schema:[test.t.a]", - " 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]", - " DataSource_1 table:t", - "Group#3 Schema:[test.t.a]", - " TopN_4 input:[Group#2], test.t.a:asc, offset:0, count:1" - ] - } - ] - }, - { - "Name": "TestMergeAggregationProjection", - "Cases": [ - { - "SQL": "select b, max(a) from (select a, c+d as b from t as t1) as t2 group by b", - "Result": [ - "Group#0 Schema:[Column#13,Column#14]", - " Projection_4 input:[Group#1], Column#13, Column#14", - "Group#1 Schema:[Column#14,Column#13]", - " Aggregation_3 input:[Group#2], group by:Column#13, funcs:max(test.t.a), firstrow(Column#13)", - " Aggregation_5 input:[Group#3], group by:plus(test.t.c, test.t.d), funcs:max(test.t.a), firstrow(plus(test.t.c, test.t.d))", - "Group#2 Schema:[test.t.a,Column#13]", - " Projection_2 input:[Group#3], test.t.a, plus(test.t.c, test.t.d)->Column#13", - "Group#3 Schema:[test.t.a,test.t.c,test.t.d]", - " DataSource_1 table:t1" - ] - }, - { - "SQL": "select max(a) from (select c+d as b, a+c as a from t as t1) as t2", - "Result": [ - "Group#0 Schema:[Column#15]", - " Projection_4 input:[Group#1], Column#15", - "Group#1 Schema:[Column#15]", - " Aggregation_3 input:[Group#2], funcs:max(Column#14)", - " Aggregation_5 input:[Group#3], funcs:max(plus(test.t.a, test.t.c))", - "Group#2 Schema:[Column#14]", - " Projection_2 input:[Group#3], plus(test.t.a, test.t.c)->Column#14", - "Group#3 Schema:[test.t.a,test.t.c]", - " DataSource_1 table:t1" - ] - }, - { - "SQL": "select b, max(a) from (select a, c+d as b, @i:=0 from t as t1) as t2 group by b", - "Result": [ - "Group#0 Schema:[Column#13,Column#15]", - " Projection_4 input:[Group#1], Column#13, Column#15", - "Group#1 Schema:[Column#15,Column#13]", - " Aggregation_3 input:[Group#2], group by:Column#13, funcs:max(test.t.a), firstrow(Column#13)", - "Group#2 Schema:[test.t.a,Column#13,Column#14]", - " Projection_2 input:[Group#3], test.t.a, plus(test.t.c, test.t.d)->Column#13, setvar(i, 0)->Column#14", - "Group#3 Schema:[test.t.a,test.t.c,test.t.d]", - " DataSource_1 table:t1" - ] - } - ] - }, - { - "Name": "TestMergeAdjacentLimit", - "Cases": [ - { - "SQL": "select b from (select b from t limit 5) as t1 limit 10", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_4 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b]", - " Projection_2 input:[Group#2], test.t.b", - "Group#2 Schema:[test.t.b]", - " Limit_6 input:[Group#3], offset:0, count:5", - "Group#3 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select b from (select b from t limit 20) as t1 limit 10", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_4 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b]", - " Projection_2 input:[Group#2], test.t.b", - "Group#2 Schema:[test.t.b]", - " Limit_6 input:[Group#3], offset:0, count:10", - "Group#3 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select b from (select b from t limit 10) as t1 limit 10", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_4 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b]", - " Projection_2 input:[Group#2], test.t.b", - "Group#2 Schema:[test.t.b]", - " Limit_6 input:[Group#3], offset:0, count:10", - "Group#3 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select b from (select b from t limit 10 offset 10) as t1 limit 10 offset 5", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_4 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b]", - " Projection_2 input:[Group#2], test.t.b", - "Group#2 Schema:[test.t.b]", - " Limit_6 input:[Group#3], offset:15, count:5", - "Group#3 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select b from (select b from t limit 10 offset 2) as t1 limit 3 offset 5", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_4 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b]", - " Projection_2 input:[Group#2], test.t.b", - "Group#2 Schema:[test.t.b]", - " Limit_6 input:[Group#3], offset:7, count:3", - "Group#3 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select b from (select b from t limit 10 offset 5) as t1 limit 5 offset 5", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_4 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b]", - " Projection_2 input:[Group#2], test.t.b", - "Group#2 Schema:[test.t.b]", - " Limit_6 input:[Group#3], offset:10, count:5", - "Group#3 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select a from (select a from t limit 3 offset 5) t1 limit 3 offset 5", - "Result": [ - "Group#0 Schema:[test.t.a]", - " Projection_4 input:[Group#1], test.t.a", - "Group#1 Schema:[test.t.a]", - " Projection_2 input:[Group#2], test.t.a", - "Group#2 Schema:[test.t.a]", - " TableDual_6 rowcount:0" - ] - } - ] - }, - { - "Name": "TestMergeAdjacentTopN", - "Cases": [ - { - "SQL": "select b from (select b from t where c > 1 order by b limit 3) as t1 order by b limit 2", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_11 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b,test.t.c]", - " TopN_14 input:[Group#2], test.t.b:asc, offset:0, count:2", - "Group#2 Schema:[test.t.b,test.t.c]", - " Selection_2 input:[Group#3], gt(test.t.c, 1)", - "Group#3 Schema:[test.t.b,test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select a from (select a from t where b > 2 order by a limit 3 offset 1) as t1 order by a limit 2 offset 1", - "Result": [ - "Group#0 Schema:[test.t.a]", - " Projection_11 input:[Group#1], test.t.a", - "Group#1 Schema:[test.t.a,test.t.b]", - " TopN_14 input:[Group#2], test.t.a:asc, offset:2, count:2", - "Group#2 Schema:[test.t.a,test.t.b]", - " Selection_2 input:[Group#3], gt(test.t.b, 2)", - "Group#3 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select * from (select * from t order by a limit 3) as t1 order by a limit 5", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Projection_10 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TopN_13 input:[Group#2], test.t.a:asc, offset:0, count:3", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select b from (select b from t order by b limit 5) as t1 order by b limit 10", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_10 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b]", - " TopN_13 input:[Group#2], test.t.b:asc, offset:0, count:5", - "Group#2 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select b from (select b from t order by b limit 20) as t1 order by b limit 10", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_10 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b]", - " TopN_13 input:[Group#2], test.t.b:asc, offset:0, count:10", - "Group#2 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select b from (select b from t order by b limit 10) as t1 order by b limit 10", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_10 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b]", - " TopN_13 input:[Group#2], test.t.b:asc, offset:0, count:10", - "Group#2 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select b from (select b from t order by b limit 10 offset 10) as t1 order by b limit 10 offset 5", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_10 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b]", - " TopN_13 input:[Group#2], test.t.b:asc, offset:15, count:5", - "Group#2 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select b from (select b from t order by b limit 10 offset 2) as t1 order by b limit 3 offset 5", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_10 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b]", - " TopN_13 input:[Group#2], test.t.b:asc, offset:7, count:3", - "Group#2 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select b from (select b from t order by b limit 10 offset 5) as t1 order by b limit 5 offset 5", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_10 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.b]", - " TopN_13 input:[Group#2], test.t.b:asc, offset:10, count:5", - "Group#2 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select a from (select a from t order by a limit 3 offset 5) as t1 order by a limit 3 offset 5", - "Result": [ - "Group#0 Schema:[test.t.a]", - " Projection_10 input:[Group#1], test.t.a", - "Group#1 Schema:[test.t.a]", - " TableDual_13 rowcount:0" - ] - }, - { - "SQL": "select b from (select b from t where c > 1 order by b, a limit 3) as t1 order by b limit 2", - "Result": [ - "Group#0 Schema:[test.t.b]", - " Projection_13 input:[Group#1], test.t.b", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c]", - " TopN_16 input:[Group#2], test.t.b:asc, test.t.a:asc, offset:0, count:2", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", - " Selection_2 input:[Group#3], gt(test.t.c, 1)", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select a from (select a from t where b > 2 order by a, b limit 3 offset 1) as t1 order by a limit 2 offset 1", - "Result": [ - "Group#0 Schema:[test.t.a]", - " Projection_13 input:[Group#1], test.t.a", - "Group#1 Schema:[test.t.a,test.t.b]", - " TopN_16 input:[Group#2], test.t.a:asc, test.t.b:asc, offset:2, count:2", - "Group#2 Schema:[test.t.a,test.t.b]", - " Selection_2 input:[Group#3], gt(test.t.b, 2)", - "Group#3 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t" - ] - } - ] - }, - { - "Name": "TestTransformLimitToTableDual", - "Cases": [ - { - "SQL": "select a from t limit 0 offset 1", - "Result": [ - "Group#0 Schema:[test.t.a]", - " TableDual_4 rowcount:0" - ] - }, - { - "SQL": "select * from t as t1 left join (select * from t limit 0 offset 1) as t2 on t1.a = t2.a;", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Projection_6 input:[Group#1], test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date, test.t.a, test.t.b, test.t.c, test.t.d, test.t.e, test.t.c_str, test.t.d_str, test.t.e_str, test.t.f, test.t.g, test.t.h, test.t.i_date", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Join_5 input:[Group#2,Group#3], left outer join, equal:[eq(test.t.a, test.t.a)]", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " DataSource_1 table:t1", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableDual_7 rowcount:0" - ] - }, - { - "SQL": "select a, b from ((select a, b from t) union all(select c as a, d as b from t limit 0 offset 5)) as t1 where a > 1", - "Result": [ - "Group#0 Schema:[Column#25,Column#26]", - " Projection_10 input:[Group#1], Column#25, Column#26", - "Group#1 Schema:[Column#25,Column#26]", - " Selection_9 input:[Group#2], gt(Column#25, 1)", - "Group#2 Schema:[Column#25,Column#26]", - " Union_6 input:[Group#3,Group#4]", - "Group#3 Schema:[Column#25,Column#26]", - " Projection_7 input:[Group#5], test.t.a, test.t.b", - "Group#5 Schema:[test.t.a,test.t.b]", - " Projection_5 input:[Group#6], test.t.a, test.t.b", - "Group#6 Schema:[test.t.a,test.t.b]", - " DataSource_4 table:t", - "Group#4 Schema:[Column#25,Column#26]", - " Projection_8 input:[Group#7], test.t.c, test.t.d", - "Group#7 Schema:[test.t.c,test.t.d]", - " TableDual_11 rowcount:0" - ] - } - ] - }, - { - "Name": "TestPostTransformationRules", - "Cases": [ - { - "SQL": "select b from (select b+10 as b from t) as t1 order by b + 10 limit 10", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_8 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13,Column#14]", - " TopN_10 input:[Group#2], Column#14:asc, offset:0, count:10", - "Group#2 Schema:[Column#13,Column#14]", - " Projection_11 input:[Group#3], plus(test.t.b, 10)->Column#13, plus(plus(test.t.b, 10), 10)->Column#14", - "Group#3 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select * from (select a+1 as c, a+b as d from t) as t1 order by c+d limit 10", - "Result": [ - "Group#0 Schema:[Column#13,Column#14]", - " Projection_8 input:[Group#1], Column#13, Column#14", - "Group#1 Schema:[Column#13,Column#14,Column#15]", - " TopN_10 input:[Group#2], Column#15:asc, offset:0, count:10", - "Group#2 Schema:[Column#13,Column#14,Column#15]", - " Projection_11 input:[Group#3], plus(test.t.a, 1)->Column#13, plus(test.t.a, test.t.b)->Column#14, plus(plus(test.t.a, 1), plus(test.t.a, test.t.b))->Column#15", - "Group#3 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select a from (select a, b from t order by b limit 10) as t1", - "Result": [ - "Group#0 Schema:[test.t.a]", - " Projection_5 input:[Group#1], test.t.a", - "Group#1 Schema:[test.t.a,test.t.b]", - " TopN_6 input:[Group#2], test.t.b:asc, offset:0, count:10", - "Group#2 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t" - ] - } - ] - }, - { - "Name": "TestPushLimitDownTiKVSingleGather", - "Cases": [ - { - "SQL": "select * from t limit 1", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_3 input:[Group#1], offset:0, count:1", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_5 input:[Group#2], table:t", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_6 input:[Group#3], offset:0, count:1", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_4 table:t, pk col:test.t.a" - ] - }, - { - "SQL": "select * from t as t1 left join (select * from t limit 2) as t2 on t1.a = t2.a;", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date,test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Join_5 input:[Group#1,Group#2], left outer join, equal:[eq(test.t.a, test.t.a)]", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_8 input:[Group#3], table:t1", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_7 table:t1, pk col:test.t.a", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_4 input:[Group#4], offset:0, count:2", - "Group#4 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TiKVSingleGather_10 input:[Group#5], table:t", - "Group#5 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " Limit_11 input:[Group#6], offset:0, count:2", - "Group#6 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.e,test.t.c_str,test.t.d_str,test.t.e_str,test.t.f,test.t.g,test.t.h,test.t.i_date]", - " TableScan_9 table:t, pk col:test.t.a" - ] - }, - { - "SQL": "select a, b from ((select a, b from t) union all(select c as a, d as b from t limit 3)) as t1 where a > 1", - "Result": [ - "Group#0 Schema:[Column#25,Column#26]", - " Selection_9 input:[Group#1], gt(Column#25, 1)", - "Group#1 Schema:[Column#25,Column#26]", - " Union_6 input:[Group#2,Group#3]", - "Group#2 Schema:[Column#25,Column#26]", - " Projection_7 input:[Group#4], test.t.a, test.t.b", - "Group#4 Schema:[test.t.a,test.t.b]", - " TiKVSingleGather_12 input:[Group#5], table:t", - "Group#5 Schema:[test.t.a,test.t.b]", - " TableScan_11 table:t, pk col:test.t.a", - "Group#3 Schema:[Column#25,Column#26]", - " Projection_8 input:[Group#6], test.t.c, test.t.d", - "Group#6 Schema:[test.t.c,test.t.d]", - " Limit_3 input:[Group#7], offset:0, count:3", - " Limit_3 input:[Group#8], offset:0, count:3", - "Group#7 Schema:[test.t.c,test.t.d]", - " TiKVSingleGather_16 input:[Group#9], table:t, index:c_d_e", - "Group#9 Schema:[test.t.c,test.t.d]", - " Limit_18 input:[Group#10], offset:0, count:3", - "Group#10 Schema:[test.t.c,test.t.d]", - " IndexScan_15 table:t, index:c, d, e", - "Group#8 Schema:[test.t.c,test.t.d]", - " TiKVSingleGather_14 input:[Group#11], table:t", - "Group#11 Schema:[test.t.c,test.t.d]", - " Limit_17 input:[Group#12], offset:0, count:3", - "Group#12 Schema:[test.t.c,test.t.d]", - " TableScan_13 table:t" - ] - } - ] - }, - { - "Name": "TestEliminateOuterJoin", - "Cases": [ - { - "SQL": "select t1.a, max(t1.b) from t as t1 left join (select * from t) as t2 on t1.a = t2.a and t1.b = 3 group by t1.a;", - "Result": [ - "Group#0 Schema:[test.t.a,Column#25]", - " Projection_6 input:[Group#1], test.t.a, Column#25", - "Group#1 Schema:[Column#25,test.t.a]", - " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", - "Group#2 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t1" - ] - }, - { - "SQL": "select t1.a, max(t1.b) from t as t1 left join (select * from t) as t2 on t1.a = t2.a group by t1.a;", - "Result": [ - "Group#0 Schema:[test.t.a,Column#25]", - " Projection_6 input:[Group#1], test.t.a, Column#25", - "Group#1 Schema:[Column#25,test.t.a]", - " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", - "Group#2 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t1" - ] - }, - { - "SQL": "select t1.a, sum(t1.h) from t as t1 left join (select * from t) as t2 on t1.h = t2.h group by t1.a;", - "Result": [ - "Group#0 Schema:[test.t.a,Column#25]", - " Projection_6 input:[Group#1], test.t.a, Column#25", - "Group#1 Schema:[Column#25,test.t.a]", - " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:sum(test.t.h), firstrow(test.t.a)", - "Group#2 Schema:[test.t.a,test.t.h]", - " Join_4 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.h, test.t.h)]", - "Group#3 Schema:[test.t.a,test.t.h]", - " DataSource_1 table:t1", - "Group#4 Schema:[test.t.h]", - " Projection_3 input:[Group#5], test.t.h", - "Group#5 Schema:[test.t.h]", - " DataSource_2 table:t" - ] - }, - { - "SQL": "select t1.a, sum(distinct t1.h) from t as t1 left join (select * from t) as t2 on t1.h = t2.h group by t1.a;", - "Result": [ - "Group#0 Schema:[test.t.a,Column#25]", - " Projection_6 input:[Group#1], test.t.a, Column#25", - "Group#1 Schema:[Column#25,test.t.a]", - " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:sum(distinct test.t.h), firstrow(test.t.a)", - "Group#2 Schema:[test.t.a,test.t.h]", - " DataSource_1 table:t1" - ] - }, - { - "SQL": "select t1.a, max(t1.b) from t as t1 left join (select * from t) as t2 on t1.a = t2.a group by t2.a;", - "Result": [ - "Group#0 Schema:[test.t.a,Column#25]", - " Projection_6 input:[Group#1], test.t.a, Column#25", - "Group#1 Schema:[Column#25,test.t.a]", - " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", - "Group#2 Schema:[test.t.a,test.t.b,test.t.a]", - " Join_4 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.a, test.t.a)]", - "Group#3 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t1", - "Group#4 Schema:[test.t.a]", - " Projection_3 input:[Group#5], test.t.a", - "Group#5 Schema:[test.t.a]", - " DataSource_2 table:t" - ] - }, - { - "SQL": "select t1.a, max(t2.b) from t as t1 left join (select * from t) as t2 on t1.a = t2.a group by t1.a;", - "Result": [ - "Group#0 Schema:[test.t.a,Column#25]", - " Projection_6 input:[Group#1], test.t.a, Column#25", - "Group#1 Schema:[Column#25,test.t.a]", - " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", - "Group#2 Schema:[test.t.a,test.t.b]", - " Join_4 input:[Group#3,Group#4], left outer join, equal:[eq(test.t.a, test.t.a)]", - "Group#3 Schema:[test.t.a]", - " DataSource_1 table:t1", - "Group#4 Schema:[test.t.a,test.t.b]", - " Projection_3 input:[Group#5], test.t.a, test.t.b", - "Group#5 Schema:[test.t.a,test.t.b]", - " DataSource_2 table:t" - ] - }, - { - "SQL": "select t2.a, max(t2.b) from t as t1 right join (select * from t) as t2 on t1.a = t2.a group by t2.a;", - "Result": [ - "Group#0 Schema:[test.t.a,Column#25]", - " Projection_6 input:[Group#1], test.t.a, Column#25", - "Group#1 Schema:[Column#25,test.t.a]", - " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", - "Group#2 Schema:[test.t.a,test.t.b]", - " Projection_3 input:[Group#3], test.t.a, test.t.b", - "Group#3 Schema:[test.t.a,test.t.b]", - " DataSource_2 table:t" - ] - }, - { - "SQL": "select t2.a, max(t2.b) from t as t1 right join (select * from t) as t2 on t1.a = t2.a and t2.c = 3 group by t2.a;", - "Result": [ - "Group#0 Schema:[test.t.a,Column#25]", - " Projection_6 input:[Group#1], test.t.a, Column#25", - "Group#1 Schema:[Column#25,test.t.a]", - " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", - " Projection_3 input:[Group#3], test.t.a, test.t.b, test.t.c", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c]", - " DataSource_2 table:t" - ] - }, - { - "SQL": "select t2.a, sum(t2.h) from t as t1 right join (select * from t) as t2 on t1.h = t2.h group by t2.a;", - "Result": [ - "Group#0 Schema:[test.t.a,Column#25]", - " Projection_6 input:[Group#1], test.t.a, Column#25", - "Group#1 Schema:[Column#25,test.t.a]", - " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:sum(test.t.h), firstrow(test.t.a)", - "Group#2 Schema:[test.t.a,test.t.h]", - " Join_4 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.h, test.t.h)]", - "Group#3 Schema:[test.t.h]", - " DataSource_1 table:t1", - "Group#4 Schema:[test.t.a,test.t.h]", - " Projection_3 input:[Group#5], test.t.a, test.t.h", - "Group#5 Schema:[test.t.a,test.t.h]", - " DataSource_2 table:t" - ] - }, - { - "SQL": "select t2.a, sum(distinct t2.h) from t as t1 right join (select * from t) as t2 on t1.h = t2.h group by t2.a;", - "Result": [ - "Group#0 Schema:[test.t.a,Column#25]", - " Projection_6 input:[Group#1], test.t.a, Column#25", - "Group#1 Schema:[Column#25,test.t.a]", - " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:sum(distinct test.t.h), firstrow(test.t.a)", - "Group#2 Schema:[test.t.a,test.t.h]", - " Projection_3 input:[Group#3], test.t.a, test.t.h", - "Group#3 Schema:[test.t.a,test.t.h]", - " DataSource_2 table:t" - ] - }, - { - "SQL": "select t2.a, max(t2.b) from t as t1 right join (select * from t) as t2 on t1.a = t2.a group by t1.a;", - "Result": [ - "Group#0 Schema:[test.t.a,Column#25]", - " Projection_6 input:[Group#1], test.t.a, Column#25", - "Group#1 Schema:[Column#25,test.t.a]", - " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", - "Group#2 Schema:[test.t.a,test.t.a,test.t.b]", - " Join_4 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.a, test.t.a)]", - "Group#3 Schema:[test.t.a]", - " DataSource_1 table:t1", - "Group#4 Schema:[test.t.a,test.t.b]", - " Projection_3 input:[Group#5], test.t.a, test.t.b", - "Group#5 Schema:[test.t.a,test.t.b]", - " DataSource_2 table:t" - ] - }, - { - "SQL": "select t2.a, max(t1.b) from t as t1 right join (select * from t) as t2 on t1.a = t2.a group by t2.a;", - "Result": [ - "Group#0 Schema:[test.t.a,Column#25]", - " Projection_6 input:[Group#1], test.t.a, Column#25", - "Group#1 Schema:[Column#25,test.t.a]", - " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", - "Group#2 Schema:[test.t.b,test.t.a]", - " Join_4 input:[Group#3,Group#4], right outer join, equal:[eq(test.t.a, test.t.a)]", - "Group#3 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t1", - "Group#4 Schema:[test.t.a]", - " Projection_3 input:[Group#5], test.t.a", - "Group#5 Schema:[test.t.a]", - " DataSource_2 table:t" - ] - }, - { - "SQL": "select t1.a, t1.b from t as t1 left join t as t2 on t1.a = t2.a and t1.b = 3 and t2.b = 3;", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b]", - " Projection_4 input:[Group#1], test.t.a, test.t.b", - "Group#1 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t1" - ] - }, - { - "SQL": "select t1.a, t1.b from t as t1 left join t as t2 on t1.b = t2.b and t1.a = 3 and t2.b = 3;", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b]", - " Projection_4 input:[Group#1], test.t.a, test.t.b", - "Group#1 Schema:[test.t.a,test.t.b]", - " Join_3 input:[Group#2,Group#3], left outer join, equal:[eq(test.t.b, test.t.b)], left cond:eq(test.t.a, 3), right cond:eq(test.t.b, 3)", - "Group#2 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t1", - "Group#3 Schema:[test.t.b]", - " DataSource_2 table:t2" - ] - }, - { - "SQL": "select t2.a, t1.b from t as t1 left join t as t2 on t1.a = t2.a and t1.a = 3 and t2.b = 3;", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b]", - " Projection_4 input:[Group#1], test.t.a, test.t.b", - "Group#1 Schema:[test.t.b,test.t.a]", - " Join_3 input:[Group#2,Group#3], left outer join, equal:[eq(test.t.a, test.t.a)], left cond:eq(test.t.a, 3), right cond:eq(test.t.b, 3)", - "Group#2 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t1", - "Group#3 Schema:[test.t.a,test.t.b]", - " DataSource_2 table:t2" - ] - }, - { - "SQL": "select t2.a, t2.b from t as t1 right join t as t2 on t1.a = t2.a and t1.b = 3 and t2.b = 3;", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b]", - " Projection_4 input:[Group#1], test.t.a, test.t.b", - "Group#1 Schema:[test.t.a,test.t.b]", - " DataSource_2 table:t2" - ] - }, - { - "SQL": "select t2.a, t2.b from t as t1 right join t as t2 on t1.b = t2.b and t1.a = 3 and t2.b = 3;", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b]", - " Projection_4 input:[Group#1], test.t.a, test.t.b", - "Group#1 Schema:[test.t.a,test.t.b]", - " Join_3 input:[Group#2,Group#3], right outer join, equal:[eq(test.t.b, test.t.b)], left cond:eq(test.t.a, 3), right cond:eq(test.t.b, 3)", - "Group#2 Schema:[test.t.a,test.t.b], UniqueKey:[test.t.a]", - " DataSource_1 table:t1", - "Group#3 Schema:[test.t.a,test.t.b]", - " DataSource_2 table:t2" - ] - }, - { - "SQL": "select t1.a, t2.b from t as t1 right join t as t2 on t1.a = t2.a and t1.a = 3 and t2.b = 3;", - "Result": [ - "Group#0 Schema:[test.t.a,test.t.b]", - " Projection_4 input:[Group#1], test.t.a, test.t.b", - "Group#1 Schema:[test.t.a,test.t.b]", - " Join_3 input:[Group#2,Group#3], right outer join, equal:[eq(test.t.a, test.t.a)], left cond:eq(test.t.a, 3), right cond:eq(test.t.b, 3)", - "Group#2 Schema:[test.t.a]", - " DataSource_1 table:t1", - "Group#3 Schema:[test.t.a,test.t.b]", - " DataSource_2 table:t2" - ] - }, - { - "SQL": "select t3.a, max(t3.b) from (select t1.a, t1.b from t as t1 left join t as t2 on t1.a = t2.a) t3 group by t3.a", - "Result": [ - "Group#0 Schema:[test.t.a,Column#25]", - " Projection_6 input:[Group#1], test.t.a, Column#25", - "Group#1 Schema:[Column#25,test.t.a]", - " Aggregation_5 input:[Group#2], group by:test.t.a, funcs:max(test.t.b), firstrow(test.t.a)", - "Group#2 Schema:[test.t.a,test.t.b]", - " Projection_4 input:[Group#3], test.t.a, test.t.b", - "Group#3 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t1" - ] - } - ] - }, - { - "Name": "TestTransformAggregateCaseToSelection", - "Cases": [ - { - "SQL": "select count(case when a > 10 then b end) from t", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_5 input:[Group#2], funcs:count(test.t.b)", - "Group#2 Schema:[test.t.a,test.t.b]", - " Selection_4 input:[Group#3], gt(test.t.a, 10)", - "Group#3 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(case when a > 10 then b end) from t group by c", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_2 input:[Group#2], group by:test.t.c, funcs:count(case(gt(test.t.a, 10), test.t.b))", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(case when a > 10 then b end) from t group by 'a'", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_2 input:[Group#2], group by:1, funcs:count(case(gt(test.t.a, 10), test.t.b))", - "Group#2 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(case when a > 10 then b end) from t group by concat(c, a)", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_2 input:[Group#2], group by:concat(cast(test.t.c), cast(test.t.a)), funcs:count(case(gt(test.t.a, 10), test.t.b))", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(case when a > 10 then b end) from t group by concat(c, 'a')", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_2 input:[Group#2], group by:concat(cast(test.t.c), \"a\"), funcs:count(case(gt(test.t.a, 10), test.t.b))", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(case when a > 10 then b end) from t group by concat('a', 'c')", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_2 input:[Group#2], group by:1, funcs:count(case(gt(test.t.a, 10), test.t.b))", - "Group#2 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(case when a > 10 then b else null end) from t", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_5 input:[Group#2], funcs:count(test.t.b)", - "Group#2 Schema:[test.t.a,test.t.b]", - " Selection_4 input:[Group#3], gt(test.t.a, 10)", - "Group#3 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(case when a > 10 then null else b end) from t", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_5 input:[Group#2], funcs:count(test.t.b)", - "Group#2 Schema:[test.t.a,test.t.b]", - " Selection_4 input:[Group#3], not(gt(test.t.a, 10))", - "Group#3 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(case when a > 10 then c else b end) 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:count(case(gt(test.t.a, 10), test.t.c, test.t.b))", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(case when a > 10 then c else 0 end) 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:count(case(gt(test.t.a, 10), test.t.c, 0))", - "Group#2 Schema:[test.t.a,test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select sum(case when a > 10 then c else 0 end) from t", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_5 input:[Group#2], funcs:sum(test.t.c)", - "Group#2 Schema:[test.t.a,test.t.c]", - " Selection_4 input:[Group#3], gt(test.t.a, 10)", - "Group#3 Schema:[test.t.a,test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select sum(case when a > 10 then c else 0.0 end) from t", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_5 input:[Group#2], funcs:sum(cast(test.t.c))", - "Group#2 Schema:[test.t.a,test.t.c]", - " Selection_4 input:[Group#3], gt(test.t.a, 10)", - "Group#3 Schema:[test.t.a,test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select sum(case when a > 10 then c else 1-1 end) from t", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_5 input:[Group#2], funcs:sum(test.t.c)", - "Group#2 Schema:[test.t.a,test.t.c]", - " Selection_4 input:[Group#3], gt(test.t.a, 10)", - "Group#3 Schema:[test.t.a,test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select sum(case when a > 0 then (case when a <= 1000 then b end) else 0 end) from t", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_7 input:[Group#2], funcs:sum(test.t.b)", - "Group#2 Schema:[test.t.a,test.t.b]", - " Selection_6 input:[Group#3], le(test.t.a, 1000)", - "Group#3 Schema:[test.t.a,test.t.b]", - " Selection_4 input:[Group#4], gt(test.t.a, 0)", - "Group#4 Schema:[test.t.a,test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select sum(case when a > 10 then 0 else c end) from t", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_5 input:[Group#2], funcs:sum(test.t.c)", - "Group#2 Schema:[test.t.a,test.t.c]", - " Selection_4 input:[Group#3], not(gt(test.t.a, 10))", - "Group#3 Schema:[test.t.a,test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select sum(case when a > 10 then 2 else 1 end) 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:sum(case(gt(test.t.a, 10), 2, 1))", - "Group#2 Schema:[test.t.a]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(DISTINCT case when a > 10 then null else c end) from t", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_5 input:[Group#2], funcs:count(distinct test.t.c)", - "Group#2 Schema:[test.t.a,test.t.c]", - " Selection_4 input:[Group#3], not(gt(test.t.a, 10))", - "Group#3 Schema:[test.t.a,test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select sum(DISTINCT case when a > 10 then c else 0 end) 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:sum(distinct case(gt(test.t.a, 10), test.t.c, 0))", - "Group#2 Schema:[test.t.a,test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select case when c > 10 then c end from t", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_2 input:[Group#1], case(gt(test.t.c, 10), test.t.c)->Column#13", - "Group#1 Schema:[test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(case when a > 10 then c end), c from t", - "Result": [ - "Group#0 Schema:[Column#13,test.t.c]", - " Projection_3 input:[Group#1], Column#13, test.t.c", - "Group#1 Schema:[Column#13,test.t.c]", - " Aggregation_2 input:[Group#2], funcs:count(case(gt(test.t.a, 10), test.t.c)), firstrow(test.t.c)", - "Group#2 Schema:[test.t.a,test.t.c]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(case when a > 10 and d < 5 then b end) from t", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_5 input:[Group#2], funcs:count(test.t.b)", - "Group#2 Schema:[test.t.a,test.t.b,test.t.d]", - " Selection_4 input:[Group#3], gt(test.t.a, 10), lt(test.t.d, 5)", - "Group#3 Schema:[test.t.a,test.t.b,test.t.d]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(case when a > 10 and d < 5 then null else b end) from t", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_5 input:[Group#2], funcs:count(test.t.b)", - "Group#2 Schema:[test.t.a,test.t.b,test.t.d]", - " Selection_4 input:[Group#3], not(and(gt(test.t.a, 10), lt(test.t.d, 5)))", - "Group#3 Schema:[test.t.a,test.t.b,test.t.d]", - " DataSource_1 table:t" - ] - } - ] - }, - { - "Name": "TestTransformAggToProj", - "Cases": [ - { - "SQL": "select count(b) from t group by a", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_5 input:[Group#1], 1->Column#13", - "Group#1 Schema:[test.t.a,test.t.b], UniqueKey:[test.t.a]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(b) from t group by b", - "Result": [ - "Group#0 Schema:[Column#13]", - " Projection_3 input:[Group#1], Column#13", - "Group#1 Schema:[Column#13]", - " Aggregation_2 input:[Group#2], group by:test.t.b, funcs:count(test.t.b)", - "Group#2 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(b) 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:count(test.t.b)", - "Group#2 Schema:[test.t.b]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select a from t group by a having sum(b) > 4", - "Result": [ - "Group#0 Schema:[test.t.a]", - " Projection_5 input:[Group#1], test.t.a", - "Group#1 Schema:[test.t.a,Column#13]", - " Selection_4 input:[Group#2], gt(Column#13, 4)", - "Group#2 Schema:[test.t.a,Column#13]", - " Projection_7 input:[Group#3], test.t.a, cast(test.t.b, decimal(65,0) BINARY)->Column#13", - "Group#3 Schema:[test.t.a,test.t.b], UniqueKey:[test.t.a]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(b), sum(b), avg(b), b, max(b), min(b), bit_and(b), bit_or(b), bit_xor(b) from t group by a having sum(b) >= 0 and count(b) >= 0 order by b", - "Result": [ - "Group#0 Schema:[Column#21,Column#22,Column#23,test.t.b,Column#25,Column#26,Column#27,Column#28,Column#29]", - " Projection_6 input:[Group#1], Column#13, Column#14, Column#15, test.t.b, Column#16, Column#17, Column#18, Column#19, Column#20", - "Group#1 Schema:[Column#13,Column#14,Column#15,test.t.b,Column#16,Column#17,Column#18,Column#19,Column#20,Column#14,Column#13]", - " Sort_5 input:[Group#2], test.t.b:asc", - "Group#2 Schema:[Column#13,Column#14,Column#15,test.t.b,Column#16,Column#17,Column#18,Column#19,Column#20,Column#14,Column#13]", - " Selection_4 input:[Group#3], ge(Column#13, 0), ge(Column#14, 0)", - "Group#3 Schema:[Column#13,Column#14,Column#15,test.t.b,Column#16,Column#17,Column#18,Column#19,Column#20,Column#14,Column#13]", - " Projection_8 input:[Group#4], 1->Column#13, cast(test.t.b, decimal(65,0) BINARY)->Column#14, cast(test.t.b, decimal(65,30) BINARY)->Column#15, test.t.b, cast(test.t.b, int(11))->Column#16, cast(test.t.b, int(11))->Column#17, ifnull(cast(test.t.b, bigint(21) UNSIGNED BINARY), 18446744073709551615)->Column#18, ifnull(cast(test.t.b, bigint(21) UNSIGNED BINARY), 0)->Column#19, ifnull(cast(test.t.b, bigint(21) UNSIGNED BINARY), 0)->Column#20, cast(test.t.b, decimal(65,0) BINARY)->Column#14, 1->Column#13", - "Group#4 Schema:[test.t.a,test.t.b], UniqueKey:[test.t.a]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(b), sum(b), avg(b), f, max(c), min(c), bit_and(c), bit_or(d), bit_xor(g) from t group by a", - "Result": [ - "Group#0 Schema:[Column#13,Column#14,Column#15,test.t.f,Column#16,Column#17,Column#18,Column#19,Column#20]", - " Projection_5 input:[Group#1], 1->Column#13, cast(test.t.b, decimal(65,0) BINARY)->Column#14, cast(test.t.b, decimal(65,30) BINARY)->Column#15, test.t.f, cast(test.t.c, int(11))->Column#16, cast(test.t.c, int(11))->Column#17, ifnull(cast(test.t.c, bigint(21) UNSIGNED BINARY), 18446744073709551615)->Column#18, ifnull(cast(test.t.d, bigint(21) UNSIGNED BINARY), 0)->Column#19, ifnull(cast(test.t.g, bigint(21) UNSIGNED BINARY), 0)->Column#20", - "Group#1 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.f,test.t.g], UniqueKey:[test.t.f,test.t.f,test.t.g,test.t.a]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(b), sum(b), avg(b), f, max(c), min(c), bit_and(c), bit_or(d), bit_xor(g), var_pop(b) from t group by a", - "Result": [ - "Group#0 Schema:[Column#13,Column#14,Column#15,test.t.f,Column#16,Column#17,Column#18,Column#19,Column#20,Column#21]", - " Projection_3 input:[Group#1], Column#13, Column#14, Column#15, test.t.f, Column#16, Column#17, Column#18, Column#19, Column#20, Column#21", - "Group#1 Schema:[Column#13,Column#14,Column#15,Column#16,Column#17,Column#18,Column#19,Column#20,Column#21,test.t.f]", - " Aggregation_2 input:[Group#2], group by:test.t.a, funcs:count(test.t.b), sum(test.t.b), avg(test.t.b), max(test.t.c), min(test.t.c), bit_and(test.t.c), bit_or(test.t.d), bit_xor(test.t.g), var_pop(test.t.b), firstrow(test.t.f)", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.f,test.t.g], UniqueKey:[test.t.f,test.t.f,test.t.g,test.t.a]", - " DataSource_1 table:t" - ] - }, - { - "SQL": "select count(b), sum(b), avg(b), f, max(c), min(c), bit_and(c), bit_or(d), bit_xor(g), group_concat(b, c, d, f) from t group by a", - "Result": [ - "Group#0 Schema:[Column#13,Column#14,Column#15,test.t.f,Column#16,Column#17,Column#18,Column#19,Column#20,Column#21]", - " Projection_3 input:[Group#1], Column#13, Column#14, Column#15, test.t.f, Column#16, Column#17, Column#18, Column#19, Column#20, Column#21", - "Group#1 Schema:[Column#13,Column#14,Column#15,Column#16,Column#17,Column#18,Column#19,Column#20,Column#21,test.t.f]", - " Aggregation_2 input:[Group#2], group by:test.t.a, funcs:count(test.t.b), sum(test.t.b), avg(test.t.b), max(test.t.c), min(test.t.c), bit_and(test.t.c), bit_or(test.t.d), bit_xor(test.t.g), group_concat(test.t.b, test.t.c, test.t.d, test.t.f separator \",\"), firstrow(test.t.f)", - "Group#2 Schema:[test.t.a,test.t.b,test.t.c,test.t.d,test.t.f,test.t.g]", - " DataSource_1 table:t" - ] - } - ] - }, - { - "Name": "TestDecorrelate", - "Cases": [ - { - "SQL": "select a from t t1 where exists (select 1 from t t2 where t1.a = t2.b)", - "Result": [ - "Group#0 Schema:[test.t.a]", - " Projection_7 input:[Group#1], test.t.a", - "Group#1 Schema:[test.t.a]", - " Apply_6 input:[Group#2,Group#3], semi join", - " Join_10 input:[Group#2,Group#4], semi join, equal:[eq(test.t.a, test.t.b)]", - "Group#2 Schema:[test.t.a]", - " DataSource_1 table:t1", - "Group#3 Schema:[test.t.b]", - " Selection_4 input:[Group#4], eq(test.t.a, test.t.b)", - "Group#4 Schema:[test.t.b]", - " DataSource_3 table:t2" - ] - } - ] - } -] From 9caa851125a05ddc641e340b24e78f382f1731c4 Mon Sep 17 00:00:00 2001 From: Feng Liyuan Date: Thu, 7 May 2020 16:22:33 +0800 Subject: [PATCH 5/9] fix test --- executor/aggregate_test.go | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 0d30a444f4518..67ecf2192fbdd 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -510,17 +510,14 @@ func (s *testSuite1) TestGroupConcatAggr(c *C) { "(PARTITION `p0` VALUES LESS THAN (2), PARTITION `p1` VALUES LESS THAN (11))") tk.MustExec("insert into ptest select * from test;") - for i := 0; i <= 1; i++ { - for j := 0; j <= 1; j++ { - tk.MustExec(fmt.Sprintf("set session tidb_opt_distinct_agg_push_down = %v", i)) - tk.MustExec(fmt.Sprintf("set session tidb_opt_agg_push_down = %v", j)) + for j := 0; j <= 1; j++ { + tk.MustExec(fmt.Sprintf("set session tidb_opt_agg_push_down = %v", j)) - result = tk.MustQuery("select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from ptest;") - result.Check(testkit.Rows("500++200++30++20++20++10 3--3--1--1--2--1")) + result = tk.MustQuery("select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from ptest;") + result.Check(testkit.Rows("500++200++30++20++20++10 3--3--1--1--2--1")) - result = tk.MustQuery("select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from ptest;") - result.Check(testkit.Rows("500,200,30,20,10")) - } + result = tk.MustQuery("select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from ptest;") + result.Check(testkit.Rows("500,200,30,20,10")) } // issue #9920 From 8a73ea1e2684836670cd5032f2cf468f6ac63d9d Mon Sep 17 00:00:00 2001 From: Feng Liyuan Date: Thu, 7 May 2020 16:24:57 +0800 Subject: [PATCH 6/9] fix test --- cmd/explaintest/r/explain.result | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/cmd/explaintest/r/explain.result b/cmd/explaintest/r/explain.result index b1d11729b88ab..ef1d5c9c1e077 100644 --- a/cmd/explaintest/r/explain.result +++ b/cmd/explaintest/r/explain.result @@ -25,29 +25,15 @@ id int(11) YES NULL drop table if exists t; create table t(id int primary key, a int, b int); explain select group_concat(a) from t group by id; -<<<<<<< HEAD id count task operator info -StreamAgg_8 8000.00 root group by:col_1, funcs:group_concat(col_0, ",") +StreamAgg_8 8000.00 root group by:col_1, funcs:group_concat(col_0 separator ",") └─Projection_18 10000.00 root cast(test.t.a), test.t.id └─TableReader_15 10000.00 root data:TableScan_14 └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:true, stats:pseudo explain select group_concat(a, b) from t group by id; id count task operator info -StreamAgg_8 8000.00 root group by:col_2, funcs:group_concat(col_0, col_1, ",") +StreamAgg_8 8000.00 root group by:col_2, funcs:group_concat(col_0, col_1 separator ",") └─Projection_18 10000.00 root cast(test.t.a), cast(test.t.b), test.t.id └─TableReader_15 10000.00 root data:TableScan_14 └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:true, stats:pseudo -======= -id estRows task access object operator info -StreamAgg_8 8000.00 root group by:Column#6, funcs:group_concat(Column#5 separator ",")->Column#4 -└─Projection_18 10000.00 root cast(test.t.a, var_string(20))->Column#5, test.t.id - └─TableReader_15 10000.00 root data:TableFullScan_14 - └─TableFullScan_14 10000.00 cop[tikv] table:t keep order:true, stats:pseudo -explain select group_concat(a, b) from t group by id; -id estRows task access object operator info -StreamAgg_8 8000.00 root group by:Column#7, funcs:group_concat(Column#5, Column#6 separator ",")->Column#4 -└─Projection_18 10000.00 root cast(test.t.a, var_string(20))->Column#5, cast(test.t.b, var_string(20))->Column#6, test.t.id - └─TableReader_15 10000.00 root data:TableFullScan_14 - └─TableFullScan_14 10000.00 cop[tikv] table:t keep order:true, stats:pseudo ->>>>>>> 7ebcc20... executor: support GROUP_CONCAT(ORDER BY) (#16591) drop table t; From 05a3a57ed3b15bbb37a8d6c1e61a88ad1a4e9599 Mon Sep 17 00:00:00 2001 From: Feng Liyuan Date: Thu, 7 May 2020 17:09:21 +0800 Subject: [PATCH 7/9] fix test --- planner/core/physical_plan_test.go | 43 +++++++++++++++++ planner/core/testdata/plan_suite_in.json | 9 ++++ planner/core/testdata/plan_suite_out.json | 59 +++++++++++++++++++++++ 3 files changed, 111 insertions(+) diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index f0d5a44e437d7..c8a6d93ecd4f0 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -15,6 +15,7 @@ package core_test import ( "context" + "fmt" . "github.com/pingcap/check" "github.com/pingcap/parser" @@ -612,7 +613,49 @@ func (s *testPlanSuite) TestUnmatchedTableInHint(c *C) { } } } +func (s *testPlanSuite) TestGroupConcatOrderby(c *C) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + s.testData.GetTestCases(c, &input, &output) + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists test;") + tk.MustExec("create table test(id int, name int)") + tk.MustExec("insert into test values(1, 10);") + tk.MustExec("insert into test values(1, 20);") + tk.MustExec("insert into test values(1, 30);") + tk.MustExec("insert into test values(2, 20);") + tk.MustExec("insert into test values(3, 200);") + tk.MustExec("insert into test values(3, 500);") + + tk.MustExec("drop table if exists ptest;") + tk.MustExec("CREATE TABLE ptest (id int,name int) PARTITION BY RANGE ( id ) " + + "(PARTITION `p0` VALUES LESS THAN (2), PARTITION `p1` VALUES LESS THAN (11))") + tk.MustExec("insert into ptest select * from test;") + tk.MustExec(fmt.Sprintf("set session tidb_opt_agg_push_down = %v", 1)) + for i, ts := range input { + s.testData.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + ts).Rows()) + output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + }) + tk.MustQuery("explain " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Check(testkit.Rows(output[i].Result...)) + } +} func (s *testPlanSuite) TestIndexJoinHint(c *C) { defer testleak.AfterTest(c)() store, dom, err := newStoreWithBootstrap() diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/testdata/plan_suite_in.json index a6aaf2907db64..d1914bc7d396a 100644 --- a/planner/core/testdata/plan_suite_in.json +++ b/planner/core/testdata/plan_suite_in.json @@ -400,6 +400,15 @@ "select /*+ TIDB_INLJ(t2) */ t1.b, t2.a from t2 t1, t2 t2 where t1.b=t2.b and t2.c=-1;" ] }, + { + "name": "TestGroupConcatOrderby", + "cases": [ + "select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from test;", + "select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from ptest;", + "select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from test;", + "select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from ptest;" + ] + }, { "name": "TestIndexJoinUnionScan", "cases": [ diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index bf60c2f418f5c..bb77589aeb2db 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -929,6 +929,65 @@ } ] }, + { + "Name": "TestGroupConcatOrderby", + "Cases": [ + { + "SQL": "select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from test;", + "Plan": [ + "StreamAgg_8 1.00 root funcs:group_concat(col_0 order by col_1 desc separator \"++\"), group_concat(col_2 order by col_3 desc, col_4 asc separator \"--\")", + "└─Projection_18 10000.00 root cast(test.test.name), test.test.name, cast(test.test.id), test.test.name, test.test.id", + " └─TableReader_15 10000.00 root data:TableScan_14", + " └─TableScan_14 10000.00 cop table:test, range:[-inf,+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "500++200++30++20++20++10 3--3--1--1--2--1" + ] + }, + { + "SQL": "select /*+ agg_to_cop */ group_concat(name ORDER BY name desc SEPARATOR '++'), group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from ptest;", + "Plan": [ + "StreamAgg_12 1.00 root funcs:group_concat(col_0 order by col_1 desc separator \"++\"), group_concat(col_2 order by col_3 desc, col_4 asc separator \"--\")", + "└─Projection_23 20000.00 root cast(test.ptest.name), test.ptest.name, cast(test.ptest.id), test.ptest.name, test.ptest.id", + " └─Union_18 20000.00 root ", + " ├─TableReader_20 10000.00 root data:TableScan_19", + " │ └─TableScan_19 10000.00 cop table:ptest, partition:p0, range:[-inf,+inf], keep order:false, stats:pseudo", + " └─TableReader_22 10000.00 root data:TableScan_21", + " └─TableScan_21 10000.00 cop table:ptest, partition:p1, range:[-inf,+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "500++200++30++20++20++10 3--3--1--1--2--1" + ] + }, + { + "SQL": "select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from test;", + "Plan": [ + "StreamAgg_8 1.00 root funcs:group_concat(distinct col_0 order by col_1 desc separator \",\")", + "└─Projection_18 10000.00 root cast(test.test.name), test.test.name", + " └─TableReader_15 10000.00 root data:TableScan_14", + " └─TableScan_14 10000.00 cop table:test, range:[-inf,+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "500,200,30,20,10" + ] + }, + { + "SQL": "select /*+ agg_to_cop */ group_concat(distinct name order by name desc) from ptest;", + "Plan": [ + "StreamAgg_12 1.00 root funcs:group_concat(distinct col_0 order by col_1 desc separator \",\")", + "└─Projection_23 20000.00 root cast(test.ptest.name), test.ptest.name", + " └─Union_18 20000.00 root ", + " ├─TableReader_20 10000.00 root data:TableScan_19", + " │ └─TableScan_19 10000.00 cop table:ptest, partition:p0, range:[-inf,+inf], keep order:false, stats:pseudo", + " └─TableReader_22 10000.00 root data:TableScan_21", + " └─TableScan_21 10000.00 cop table:ptest, partition:p1, range:[-inf,+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "500,200,30,20,10" + ] + } + ] + }, { "Name": "TestIndexJoinUnionScan", "Cases": [ From bef0c2cb2ad88bc5c257e0572a9ef01f294b78c3 Mon Sep 17 00:00:00 2001 From: Feng Liyuan Date: Thu, 7 May 2020 17:14:57 +0800 Subject: [PATCH 8/9] fix test --- types/datum.go | 44 ++++++++++++++++++++++++++++++++++++++++++++ types/datum_test.go | 2 ++ 2 files changed, 46 insertions(+) diff --git a/types/datum.go b/types/datum.go index 4e17f14319af4..4fa5b4a608775 100644 --- a/types/datum.go +++ b/types/datum.go @@ -342,6 +342,50 @@ func (d *Datum) SetAutoID(id int64, flag uint) { } } +// String returns a human-readable description of Datum. It is intended only for debugging. +func (d Datum) String() string { + var t string + switch d.k { + case KindNull: + t = "KindNull" + case KindInt64: + t = "KindInt64" + case KindUint64: + t = "KindUint64" + case KindFloat32: + t = "KindFloat32" + case KindFloat64: + t = "KindFloat64" + case KindString: + t = "KindString" + case KindBytes: + t = "KindBytes" + case KindMysqlDecimal: + t = "KindMysqlDecimal" + case KindMysqlDuration: + t = "KindMysqlDuration" + case KindMysqlEnum: + t = "KindMysqlEnum" + case KindBinaryLiteral: + t = "KindBinaryLiteral" + case KindMysqlBit: + t = "KindMysqlBit" + case KindMysqlSet: + t = "KindMysqlSet" + case KindMysqlJSON: + t = "KindMysqlJSON" + case KindMysqlTime: + t = "KindMysqlTime" + default: + t = "Unknown" + } + v := d.GetValue() + if b, ok := v.([]byte); ok && d.k == KindBytes { + v = string(b) + } + return fmt.Sprintf("%v %v", t, v) +} + // GetValue gets the value of the datum of any kind. func (d *Datum) GetValue() interface{} { switch d.k { diff --git a/types/datum_test.go b/types/datum_test.go index d525082990ad7..4e2d464ad0eda 100644 --- a/types/datum_test.go +++ b/types/datum_test.go @@ -14,6 +14,7 @@ package types import ( + "fmt" "reflect" "testing" "time" @@ -47,6 +48,7 @@ func (ts *testDatumSuite) TestDatum(c *C) { d.SetCollation(d.Collation()) c.Assert(d.Collation(), NotNil) c.Assert(d.Length(), Equals, int(d.length)) + c.Assert(fmt.Sprint(d), Equals, d.String()) } } From 7247df1dfa45da7e1a44ea6f5e7b239e064b8b67 Mon Sep 17 00:00:00 2001 From: Feng Liyuan Date: Thu, 7 May 2020 17:22:20 +0800 Subject: [PATCH 9/9] fix vet --- planner/core/rule_max_min_eliminate.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/rule_max_min_eliminate.go b/planner/core/rule_max_min_eliminate.go index cb970a1516d4d..b67980c4bb712 100644 --- a/planner/core/rule_max_min_eliminate.go +++ b/planner/core/rule_max_min_eliminate.go @@ -181,7 +181,7 @@ func (a *maxMinEliminator) eliminateSingleMaxMin(agg *LogicalAggregation) *Logic desc := f.Name == ast.AggFuncMax // Compose Sort operator. sort := LogicalSort{}.Init(ctx) - sort.ByItems = append(sort.ByItems, &util.ByItems{f.Args[0], desc}) + sort.ByItems = append(sort.ByItems, &util.ByItems{Expr: f.Args[0], Desc: desc}) sort.SetChildren(child) child = sort }