diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index 7a05282ebc9ed..3592aede609fd 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -15,12 +15,17 @@ package executor import ( + "cmp" "context" "encoding/base64" "fmt" "math/rand" "os" +<<<<<<< HEAD:executor/benchmark_test.go "sort" +======= + "slices" +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/executor/benchmark_test.go "strconv" "strings" "sync" @@ -882,7 +887,49 @@ func defaultHashJoinTestCase(cols []*types.FieldType, joinType core.JoinType, us return tc } +<<<<<<< HEAD:executor/benchmark_test.go func prepare4HashJoin(testCase *hashJoinTestCase, innerExec, outerExec Executor) *HashJoinExec { +======= +func prepareResolveIndices(joinSchema, lSchema, rSchema *expression.Schema, joinType core.JoinType) *expression.Schema { + colsNeedResolving := joinSchema.Len() + // The last output column of this two join is the generated column to indicate whether the row is matched or not. + if joinType == core.LeftOuterSemiJoin || joinType == core.AntiLeftOuterSemiJoin { + colsNeedResolving-- + } + mergedSchema := expression.MergeSchema(lSchema, rSchema) + // To avoid that two plan shares the same column slice. + shallowColSlice := make([]*expression.Column, joinSchema.Len()) + copy(shallowColSlice, joinSchema.Columns) + joinSchema = expression.NewSchema(shallowColSlice...) + foundCnt := 0 + // Here we want to resolve all join schema columns directly as a merged schema, and you know same name + // col in join schema should be separately redirected to corresponded same col in child schema. But two + // column sets are **NOT** always ordered, see comment: https://github.com/pingcap/tidb/pull/45831#discussion_r1481031471 + // we are using mapping mechanism instead of moving j forward. + marked := make([]bool, mergedSchema.Len()) + for i := 0; i < colsNeedResolving; i++ { + findIdx := -1 + for j := 0; j < len(mergedSchema.Columns); j++ { + if !joinSchema.Columns[i].EqualColumn(mergedSchema.Columns[j]) || marked[j] { + continue + } + // resolve to a same unique id one, and it not being marked. + findIdx = j + break + } + if findIdx != -1 { + // valid one. + joinSchema.Columns[i] = joinSchema.Columns[i].Clone().(*expression.Column) + joinSchema.Columns[i].Index = findIdx + marked[findIdx] = true + foundCnt++ + } + } + return joinSchema +} + +func prepare4HashJoin(testCase *hashJoinTestCase, innerExec, outerExec exec.Executor) *HashJoinExec { +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/executor/benchmark_test.go if testCase.useOuterToBuild { innerExec, outerExec = outerExec, innerExec } @@ -960,26 +1007,57 @@ func prepare4HashJoin(testCase *hashJoinTestCase, innerExec, outerExec Executor) // markChildrenUsedColsForTest compares each child with the output schema, and mark // each column of the child is used by output or not. +<<<<<<< HEAD:executor/benchmark_test.go func markChildrenUsedColsForTest(outputSchema *expression.Schema, childSchemas ...*expression.Schema) (childrenUsed [][]bool) { childrenUsed = make([][]bool, 0, len(childSchemas)) markedOffsets := make(map[int]struct{}) for _, col := range outputSchema.Columns { markedOffsets[col.Index] = struct{}{} +======= +func markChildrenUsedColsForTest(ctx sessionctx.Context, outputSchema *expression.Schema, childSchemas ...*expression.Schema) (childrenUsed [][]int) { + childrenUsed = make([][]int, 0, len(childSchemas)) + markedOffsets := make(map[int]int) + for originalIdx, col := range outputSchema.Columns { + markedOffsets[col.Index] = originalIdx +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/executor/benchmark_test.go } prefixLen := 0 + type intPair struct { + first int + second int + } + // for example here. + // left child schema: [col11] + // right child schema: [col21, col22] + // output schema is [col11, col22, col21], if not records the original derived order after physical resolve index. + // the lused will be [0], the rused will be [0,1], while the actual order is dismissed, [1,0] is correct for rused. for _, childSchema := range childSchemas { - used := make([]bool, len(childSchema.Columns)) + usedIdxPair := make([]intPair, 0, len(childSchema.Columns)) for i := range childSchema.Columns { - if _, ok := markedOffsets[prefixLen+i]; ok { - used[i] = true + if originalIdx, ok := markedOffsets[prefixLen+i]; ok { + usedIdxPair = append(usedIdxPair, intPair{first: originalIdx, second: i}) } } +<<<<<<< HEAD:executor/benchmark_test.go childrenUsed = append(childrenUsed, used) } for _, child := range childSchemas { used := expression.GetUsedList(outputSchema.Columns, child) childrenUsed = append(childrenUsed, used) } +======= + // sort the used idxes according their original indexes derived after resolveIndex. + slices.SortFunc(usedIdxPair, func(a, b intPair) int { + return cmp.Compare(a.first, b.first) + }) + usedIdx := make([]int, 0, len(childSchema.Columns)) + for _, one := range usedIdxPair { + usedIdx = append(usedIdx, one.second) + } + childrenUsed = append(childrenUsed, usedIdx) + prefixLen += childSchema.Len() + } +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/executor/benchmark_test.go return } @@ -1580,15 +1658,35 @@ func prepareMergeJoinExec(tc *mergeJoinTestCase, joinSchema *expression.Schema, isOuterJoin: false, } + var usedIdx [][]int + if tc.childrenUsedSchema != nil { + usedIdx = make([][]int, 0, len(tc.childrenUsedSchema)) + for _, childSchema := range tc.childrenUsedSchema { + used := make([]int, 0, len(childSchema)) + for idx, one := range childSchema { + if one { + used = append(used, idx) + } + } + usedIdx = append(usedIdx, used) + } + } + mergeJoinExec.joiner = newJoiner( tc.ctx, 0, false, defaultValues, nil, +<<<<<<< HEAD:executor/benchmark_test.go retTypes(leftExec), retTypes(rightExec), tc.childrenUsedSchema, +======= + exec.RetTypes(leftExec), + exec.RetTypes(rightExec), + usedIdx, +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/executor/benchmark_test.go false, ) diff --git a/executor/builder.go b/executor/builder.go index 83a932d1e1064..7a19f61f4b094 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -716,14 +716,11 @@ func (b *executorBuilder) buildLimit(v *plannercore.PhysicalLimit) Executor { end: v.Offset + v.Count, } + childUsedSchemaLen := v.Children()[0].Schema().Len() childUsedSchema := markChildrenUsedCols(v.Schema().Columns, v.Children()[0].Schema())[0] e.columnIdxsUsedByChild = make([]int, 0, len(childUsedSchema)) - for i, used := range childUsedSchema { - if used { - e.columnIdxsUsedByChild = append(e.columnIdxsUsedByChild, i) - } - } - if len(e.columnIdxsUsedByChild) == len(childUsedSchema) { + e.columnIdxsUsedByChild = append(e.columnIdxsUsedByChild, childUsedSchema...) + if len(e.columnIdxsUsedByChild) == childUsedSchemaLen { e.columnIdxsUsedByChild = nil // indicates that all columns are used. LimitExec will improve performance for this condition. } return e @@ -2993,21 +2990,39 @@ func constructDistExec(sctx sessionctx.Context, plans []plannercore.PhysicalPlan // markChildrenUsedCols compares each child with the output schema, and mark // each column of the child is used by output or not. -func markChildrenUsedCols(outputCols []*expression.Column, childSchemas ...*expression.Schema) (childrenUsed [][]bool) { - childrenUsed = make([][]bool, 0, len(childSchemas)) - markedOffsets := make(map[int]struct{}) - for _, col := range outputCols { - markedOffsets[col.Index] = struct{}{} +func markChildrenUsedCols(outputCols []*expression.Column, childSchemas ...*expression.Schema) (childrenUsed [][]int) { + childrenUsed = make([][]int, 0, len(childSchemas)) + markedOffsets := make(map[int]int) + // keep the original maybe reversed order. + for originalIdx, col := range outputCols { + markedOffsets[col.Index] = originalIdx } prefixLen := 0 + type intPair struct { + first int + second int + } + // for example here. + // left child schema: [col11] + // right child schema: [col21, col22] + // output schema is [col11, col22, col21], if not records the original derived order after physical resolve index. + // the lused will be [0], the rused will be [0,1], while the actual order is dismissed, [1,0] is correct for rused. for _, childSchema := range childSchemas { - used := make([]bool, len(childSchema.Columns)) + usedIdxPair := make([]intPair, 0, len(childSchema.Columns)) for i := range childSchema.Columns { - if _, ok := markedOffsets[prefixLen+i]; ok { - used[i] = true + if originalIdx, ok := markedOffsets[prefixLen+i]; ok { + usedIdxPair = append(usedIdxPair, intPair{first: originalIdx, second: i}) } } - childrenUsed = append(childrenUsed, used) + // sort the used idxes according their original indexes derived after resolveIndex. + slices.SortFunc(usedIdxPair, func(a, b intPair) int { + return cmp.Compare(a.first, b.first) + }) + usedIdx := make([]int, 0, len(childSchema.Columns)) + for _, one := range usedIdxPair { + usedIdx = append(usedIdx, one.second) + } + childrenUsed = append(childrenUsed, usedIdx) prefixLen += childSchema.Len() } return diff --git a/executor/joiner.go b/executor/joiner.go index 842135802444f..4bcc4d042fbd9 100644 --- a/executor/joiner.go +++ b/executor/joiner.go @@ -132,7 +132,7 @@ func JoinerType(j joiner) plannercore.JoinType { func newJoiner(ctx sessionctx.Context, joinType plannercore.JoinType, outerIsRight bool, defaultInner []types.Datum, filter []expression.Expression, - lhsColTypes, rhsColTypes []*types.FieldType, childrenUsed [][]bool, isNA bool) joiner { + lhsColTypes, rhsColTypes []*types.FieldType, childrenUsed [][]int, isNA bool) joiner { base := baseJoiner{ ctx: ctx, conditions: filter, @@ -141,19 +141,14 @@ func newJoiner(ctx sessionctx.Context, joinType plannercore.JoinType, } base.selected = make([]bool, 0, chunk.InitialCapacity) base.isNull = make([]bool, 0, chunk.InitialCapacity) + // lused and rused should be followed with its original order. + // the case is that is join schema rely on the reversed order + // of child's schema, here we should keep it original order. if childrenUsed != nil { base.lUsed = make([]int, 0, len(childrenUsed[0])) // make it non-nil - for i, used := range childrenUsed[0] { - if used { - base.lUsed = append(base.lUsed, i) - } - } + base.lUsed = append(base.lUsed, childrenUsed[0]...) base.rUsed = make([]int, 0, len(childrenUsed[1])) // make it non-nil - for i, used := range childrenUsed[1] { - if used { - base.rUsed = append(base.rUsed, i) - } - } + base.rUsed = append(base.rUsed, childrenUsed[1]...) logutil.BgLogger().Debug("InlineProjection", zap.Ints("lUsed", base.lUsed), zap.Ints("rUsed", base.rUsed), zap.Int("lCount", len(lhsColTypes)), zap.Int("rCount", len(rhsColTypes))) diff --git a/executor/tiflashtest/tiflash_test.go b/executor/tiflashtest/tiflash_test.go index d52049a0f5b38..d2a2dbe4599c3 100644 --- a/executor/tiflashtest/tiflash_test.go +++ b/executor/tiflashtest/tiflash_test.go @@ -1326,6 +1326,126 @@ func TestTiflashEmptyDynamicPruneResult(t *testing.T) { tk.MustQuery("select /*+ read_from_storage(tiflash[t1, t2]) */ * from IDT_RP24833 partition(p3, p4) t1 join IDT_RP24833 partition(p2) t2 on t1.col1 = t2.col1 where t1. col1 between -8448770111093677011 and -8448770111093677011 and t2. col1 <= -8448770111093677011;").Check(testkit.Rows()) } +<<<<<<< HEAD:executor/tiflashtest/tiflash_test.go +======= +func TestDisaggregatedTiFlash(t *testing.T) { + config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = true + conf.UseAutoScaler = true + }) + defer config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = false + conf.UseAutoScaler = false + }) + err := tiflashcompute.InitGlobalTopoFetcher(tiflashcompute.TestASStr, "tmpAddr", "tmpClusterID", false) + require.NoError(t, err) + + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(c1 int)") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + + err = tk.ExecToErr("select * from t;") + // Expect error, because TestAutoScaler return empty topo. + require.Contains(t, err.Error(), "Cannot find proper topo to dispatch MPPTask: topo from AutoScaler is empty") + + err = tiflashcompute.InitGlobalTopoFetcher(tiflashcompute.AWSASStr, "tmpAddr", "tmpClusterID", false) + require.NoError(t, err) + err = tk.ExecToErr("select * from t;") + // Expect error, because AWSAutoScaler is not setup, so http request will fail. + require.Contains(t, err.Error(), "[util:1815]Internal : get tiflash_compute topology failed") +} + +// todo: remove this after AutoScaler is stable. +func TestDisaggregatedTiFlashNonAutoScaler(t *testing.T) { + config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = true + conf.UseAutoScaler = false + }) + defer config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = false + conf.UseAutoScaler = true + }) + + // Setting globalTopoFetcher to nil to can make sure cannot fetch topo from AutoScaler. + err := tiflashcompute.InitGlobalTopoFetcher(tiflashcompute.InvalidASStr, "tmpAddr", "tmpClusterID", false) + require.Contains(t, err.Error(), "unexpected topo fetch type. expect: mock or aws or gcp, got invalid") + + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(c1 int)") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + + err = tk.ExecToErr("select * from t;") + // This error message means we use PD instead of AutoScaler. + require.Contains(t, err.Error(), "tiflash_compute node is unavailable") +} + +func TestDisaggregatedTiFlashQuery(t *testing.T) { + config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = true + }) + defer config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = false + }) + + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists tbl_1") + tk.MustExec(`create table tbl_1 ( col_1 bigint not null default -1443635317331776148, + col_2 text ( 176 ) collate utf8mb4_bin not null, + col_3 decimal ( 8, 3 ), + col_4 varchar ( 128 ) collate utf8mb4_bin not null, + col_5 varchar ( 377 ) collate utf8mb4_bin, + col_6 double, + col_7 varchar ( 459 ) collate utf8mb4_bin, + col_8 tinyint default -88 ) charset utf8mb4 collate utf8mb4_bin ;`) + tk.MustExec("alter table tbl_1 set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "tbl_1") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + + tk.MustExec("explain select max( tbl_1.col_1 ) as r0 , sum( tbl_1.col_1 ) as r1 , sum( tbl_1.col_8 ) as r2 from tbl_1 where tbl_1.col_8 != 68 or tbl_1.col_3 between null and 939 order by r0,r1,r2;") + + tk.MustExec("set @@tidb_partition_prune_mode = 'static';") + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + tk.MustExec("create table t1(c1 int, c2 int) partition by hash(c1) partitions 3") + tk.MustExec("insert into t1 values(1, 1), (2, 2), (3, 3)") + tk.MustExec("alter table t1 set tiflash replica 1") + tb = external.GetTableByName(t, tk, "test", "t1") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustQuery("explain select * from t1 where c1 < 2").Check(testkit.Rows( + "PartitionUnion_11 9970.00 root ", + "├─TableReader_16 3323.33 root MppVersion: 2, data:ExchangeSender_15", + "│ └─ExchangeSender_15 3323.33 mpp[tiflash] ExchangeType: PassThrough", + "│ └─Selection_14 3323.33 mpp[tiflash] lt(test.t1.c1, 2)", + "│ └─TableFullScan_13 10000.00 mpp[tiflash] table:t1, partition:p0 pushed down filter:empty, keep order:false, stats:pseudo", + "├─TableReader_20 3323.33 root MppVersion: 2, data:ExchangeSender_19", + "│ └─ExchangeSender_19 3323.33 mpp[tiflash] ExchangeType: PassThrough", + "│ └─Selection_18 3323.33 mpp[tiflash] lt(test.t1.c1, 2)", + "│ └─TableFullScan_17 10000.00 mpp[tiflash] table:t1, partition:p1 pushed down filter:empty, keep order:false, stats:pseudo", + "└─TableReader_24 3323.33 root MppVersion: 2, data:ExchangeSender_23", + " └─ExchangeSender_23 3323.33 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection_22 3323.33 mpp[tiflash] lt(test.t1.c1, 2)", + " └─TableFullScan_21 10000.00 mpp[tiflash] table:t1, partition:p2 pushed down filter:empty, keep order:false, stats:pseudo")) +} + +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/executor/test/tiflashtest/tiflash_test.go func TestMPPMemoryTracker(t *testing.T) { store := testkit.CreateMockStore(t, withMockTiFlash(2)) tk := testkit.NewTestKit(t, store) diff --git a/pkg/planner/core/casetest/testdata/integration_suite_out.json b/pkg/planner/core/casetest/testdata/integration_suite_out.json new file mode 100644 index 0000000000000..fff25a3288389 --- /dev/null +++ b/pkg/planner/core/casetest/testdata/integration_suite_out.json @@ -0,0 +1,1220 @@ +[ + { + "Name": "TestIssue31240", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from t31240;", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(test.t31240._tidb_rowid)->Column#6", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t31240 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "set @@tidb_isolation_read_engines=\"tiflash,tidb\";", + "Plan": null + }, + { + "SQL": "explain format = 'brief' select count(*) from t31240;", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(test.t31240._tidb_rowid)->Column#6", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t31240 keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestVerboseExplain", + "Cases": [ + { + "SQL": "explain format = 'verbose' select count(*) from t3", + "Plan": [ + "StreamAgg_20 1.00 102.69 root funcs:count(Column#9)->Column#4", + "└─IndexReader_21 1.00 52.79 root index:StreamAgg_8", + " └─StreamAgg_8 1.00 760.20 cop[tikv] funcs:count(1)->Column#9", + " └─IndexFullScan_19 3.00 610.50 cop[tikv] table:t3, index:c(b) keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t2", + "Plan": [ + "StreamAgg_27 1.00 107.45 root funcs:count(Column#7)->Column#4", + "└─TableReader_28 1.00 57.55 root data:StreamAgg_10", + " └─StreamAgg_10 1.00 831.62 cop[tikv] funcs:count(1)->Column#7", + " └─TableFullScan_25 3.00 681.92 cop[tikv] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select * from t3 order by a", + "Plan": [ + "Sort_4 3.00 318.27 root test.t3.a", + "└─TableReader_8 3.00 70.81 root data:TableFullScan_7", + " └─TableFullScan_7 3.00 681.92 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select * from t3 order by b", + "Plan": [ + "Sort_4 3.00 318.27 root test.t3.b", + "└─TableReader_8 3.00 70.81 root data:TableFullScan_7", + " └─TableFullScan_7 3.00 681.92 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select * from t3 order by a limit 1", + "Plan": [ + "TopN_7 1.00 53.10 root test.t3.a, offset:0, count:1", + "└─TableReader_16 1.00 49.90 root data:TopN_15", + " └─TopN_15 1.00 685.12 cop[tikv] test.t3.a, offset:0, count:1", + " └─TableFullScan_14 3.00 681.92 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select * from t3 order by b limit 1", + "Plan": [ + "TopN_7 1.00 53.10 root test.t3.b, offset:0, count:1", + "└─TableReader_16 1.00 49.90 root data:TopN_15", + " └─TopN_15 1.00 685.12 cop[tikv] test.t3.b, offset:0, count:1", + " └─TableFullScan_14 3.00 681.92 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t2 group by a", + "Plan": [ + "HashAgg_8 3.00 1706.09 root group by:test.t2.a, funcs:count(1)->Column#4", + "└─TableReader_17 3.00 58.13 root data:TableFullScan_16", + " └─TableFullScan_16 3.00 681.92 cop[tikv] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t3 where b = 0", + "Plan": [ + "StreamAgg_10 1.00 64.98 root funcs:count(1)->Column#4", + "└─IndexReader_15 0.00 15.08 root index:IndexRangeScan_14", + " └─IndexRangeScan_14 0.00 162.80 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select /*+ use_index(t3, c) */ count(a) from t3 where b = 0", + "Plan": [ + "StreamAgg_10 1.00 2001.63 root funcs:count(test.t3.a)->Column#4", + "└─IndexLookUp_17 0.00 1951.73 root ", + " ├─IndexRangeScan_15(Build) 0.00 203.50 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false", + " └─TableRowIDScan_16(Probe) 0.00 227.31 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t2 where a = 0", + "Plan": [ + "StreamAgg_12 1.00 109.57 root funcs:count(1)->Column#4", + "└─TableReader_20 0.00 59.67 root data:Selection_19", + " └─Selection_19 0.00 831.62 cop[tikv] eq(test.t2.a, 0)", + " └─TableFullScan_18 3.00 681.92 cop[tikv] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t3 t join t3 on t.a = t3.b", + "Plan": [ + "StreamAgg_10 1.00 2128.93 root funcs:count(1)->Column#7", + "└─HashJoin_40 3.00 1979.23 root inner join, equal:[eq(test.t3.a, test.t3.b)]", + " ├─IndexReader_28(Build) 3.00 45.23 root index:IndexFullScan_27", + " │ └─IndexFullScan_27 3.00 488.40 cop[tikv] table:t3, index:c(b) keep order:false", + " └─TableReader_26(Probe) 3.00 68.11 root data:Selection_25", + " └─Selection_25 3.00 831.62 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan_24 3.00 681.92 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", + "Plan": [ + "StreamAgg_15 1.00 62053.22 root funcs:count(1)->Column#7", + "└─TableReader_43 3.00 61903.52 root MppVersion: 2, data:ExchangeSender_42", + " └─ExchangeSender_42 3.00 928447.20 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_39 3.00 928447.20 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_22(Build) 3.00 464290.40 mpp[tiflash] ", + " │ └─ExchangeSender_21 3.00 464146.40 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_20 3.00 464146.40 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan_19 3.00 464139.20 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false", + " └─Selection_24(Probe) 3.00 464146.40 mpp[tiflash] not(isnull(test.t2.a))", + " └─TableFullScan_23 3.00 464139.20 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", + "Plan": [ + "StreamAgg_15 1.00 71713.64 root funcs:count(1)->Column#10", + "└─HashJoin_61 3.00 71563.94 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─IndexReader_49(Build) 3.00 45.23 root index:IndexFullScan_48", + " │ └─IndexFullScan_48 3.00 488.40 cop[tikv] table:t3, index:c(b) keep order:false", + " └─TableReader_39(Probe) 3.00 69652.83 root MppVersion: 2, data:ExchangeSender_38", + " └─ExchangeSender_38 3.00 1044634.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_29 3.00 1044634.00 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_35(Build) 3.00 580476.40 mpp[tiflash] ", + " │ └─ExchangeSender_34 3.00 580188.40 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_33 3.00 580188.40 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan_32 3.00 580174.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false", + " └─Selection_37(Probe) 3.00 464146.40 mpp[tiflash] not(isnull(test.t2.a))", + " └─TableFullScan_36 3.00 464139.20 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select (2) in (select /*+ read_from_storage(tiflash[t1]) */ count(*) from t1) from (select t.b < (select /*+ read_from_storage(tiflash[t2]) */ t.b from t2 limit 1 ) from t3 t) t; -- we do generate the agg pushed-down plan of mpp, but cost-cmp failed", + "Plan": [ + "HashJoin_17 3.00 32771.06 root CARTESIAN left outer semi join", + "├─Selection_23(Build) 0.80 31149.25 root eq(2, Column#18)", + "│ └─StreamAgg_30 1.00 31099.35 root funcs:count(1)->Column#18", + "│ └─TableReader_42 3.00 30949.65 root MppVersion: 2, data:ExchangeSender_41", + "│ └─ExchangeSender_41 3.00 464139.20 mpp[tiflash] ExchangeType: PassThrough", + "│ └─TableFullScan_40 3.00 464139.20 mpp[tiflash] table:t1 keep order:false", + "└─Projection_18(Probe) 3.00 53.67 root 1->Column#23", + " └─IndexReader_22 3.00 53.37 root index:IndexFullScan_21", + " └─IndexFullScan_21 3.00 610.50 cop[tikv] table:t, index:c(b) keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select /*+ merge_join(t1), read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", + "Plan": [ + "StreamAgg_15 1.00 62546.70 root funcs:count(1)->Column#7", + "└─MergeJoin_29 3.00 62397.00 root inner join, left key:test.t1.a, right key:test.t2.a", + " ├─Sort_27(Build) 3.00 31197.00 root test.t2.a", + " │ └─TableReader_26 3.00 30950.13 root MppVersion: 2, data:ExchangeSender_25", + " │ └─ExchangeSender_25 3.00 464146.40 mpp[tiflash] ExchangeType: PassThrough", + " │ └─Selection_24 3.00 464146.40 mpp[tiflash] not(isnull(test.t2.a))", + " │ └─TableFullScan_23 3.00 464139.20 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false", + " └─Sort_22(Probe) 3.00 31197.00 root test.t1.a", + " └─TableReader_21 3.00 30950.13 root MppVersion: 2, data:ExchangeSender_20", + " └─ExchangeSender_20 3.00 464146.40 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection_19 3.00 464146.40 mpp[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan_18 3.00 464139.20 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false" + ] + } + ] + }, + { + "Name": "TestIsolationReadDoNotFilterSystemDB", + "Cases": [ + { + "SQL": "desc format = 'brief' select * from metrics_schema.tidb_query_duration where time >= '2019-12-23 16:10:13' and time <= '2019-12-23 16:30:13'", + "Plan": [ + "MemTableScan 10000.00 root table:tidb_query_duration PromQL:histogram_quantile(0.9, sum(rate(tidb_server_handle_query_duration_seconds_bucket{}[60s])) by (le,sql_type,instance)), start_time:2019-12-23 16:10:13, end_time:2019-12-23 16:30:13, step:1m0s" + ] + }, + { + "SQL": "desc format = 'brief' select * from information_schema.tables", + "Plan": [ + "MemTableScan 10000.00 root table:TABLES " + ] + }, + { + "SQL": "desc format = 'brief' select * from mysql.stats_meta", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:stats_meta keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIsolationReadTiFlashNotChoosePointGet", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t where t.a = 1", + "Result": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableRangeScan 1.00 mpp[tiflash] table:t range:[1,1], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where t.a in (1, 2)", + "Result": [ + "TableReader 2.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableRangeScan 2.00 mpp[tiflash] table:t range:[1,1], [2,2], keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestMergeContinuousSelections", + "Cases": [ + { + "SQL": "desc format = 'brief' SELECT table2 . `col_char_64` AS field1 FROM `ts` AS table2 INNER JOIN (SELECT DISTINCT SUBQUERY3_t1 . * FROM `ts` AS SUBQUERY3_t1 LEFT OUTER JOIN `ts` AS SUBQUERY3_t2 ON SUBQUERY3_t2 . `col_varchar_64_not_null` = SUBQUERY3_t1 . `col_varchar_key`) AS table3 ON (table3 . `col_varchar_key` = table2 . `col_varchar_64`) WHERE table3 . `col_char_64_not_null` >= SOME (SELECT SUBQUERY4_t1 . `col_varchar_64` AS SUBQUERY4_field1 FROM `ts` AS SUBQUERY4_t1) GROUP BY field1 ;", + "Plan": [ + "HashAgg 7992.00 root group by:test.ts.col_char_64, funcs:firstrow(test.ts.col_char_64)->test.ts.col_char_64", + "└─HashJoin 9990.00 root CARTESIAN inner join, other cond:or(ge(test.ts.col_char_64_not_null, Column#25), if(ne(Column#26, 0), NULL, 0))", + " ├─Selection(Build) 0.80 root ne(Column#27, 0)", + " │ └─HashAgg 1.00 root funcs:min(Column#36)->Column#25, funcs:sum(Column#37)->Column#26, funcs:count(Column#38)->Column#27", + " │ └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 mpp[tiflash] funcs:min(Column#42)->Column#36, funcs:sum(Column#43)->Column#37, funcs:count(1)->Column#38", + " │ └─Projection 10000.00 mpp[tiflash] test.ts.col_varchar_64->Column#42, cast(isnull(test.ts.col_varchar_64), decimal(20,0) BINARY)->Column#43", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:SUBQUERY4_t1 keep order:false, stats:pseudo", + " └─TableReader(Probe) 12487.50 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.ts.col_varchar_64, test.ts.col_varchar_key)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.ts.col_varchar_64))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:table2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.ts.col_varchar_key))", + " └─TableFullScan 10000.00 mpp[tiflash] table:SUBQUERY3_t1 pushed down filter:empty, keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestPushDownGroupConcatToTiFlash", + "Cases": [ + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts", + "Plan": [ + "HashAgg 1.00 root funcs:group_concat(Column#6 separator \",\")->Column#5", + "└─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#8, Column#9, Column#10 separator \",\")->Column#6", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#8, test.ts.col_1->Column#9, cast(test.ts.id, var_string(20))->Column#10", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#6, Column#7, Column#8 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0->Column#6, test.ts.col_1->Column#7, cast(test.ts.id, var_string(20))->Column#8", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#6, test.ts.col_1->Column#7, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0->Column#9", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0->Column#6, test.ts.col_1->Column#7, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0->Column#9", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(*),min(col_1) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#8, Column#9, Column#10 order by Column#11 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:min(Column#12)->Column#7", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#8, test.ts.col_1->Column#9, cast(test.ts.id, var_string(20))->Column#10, test.ts.col_0->Column#11, test.ts.col_1->Column#12", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(*),max(col_0) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#12, Column#13, Column#14 order by Column#15 separator \",\")->Column#5, funcs:sum(Column#16)->Column#6, funcs:max(Column#17)->Column#7", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0->Column#12, test.ts.col_1->Column#13, cast(test.ts.id, var_string(20))->Column#14, test.ts.col_0->Column#15, Column#10->Column#16, Column#11->Column#17", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, funcs:count(1)->Column#10, funcs:max(test.ts.col_0)->Column#11", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#12, funcs:group_concat(Column#9, Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#9, test.ts.col_1->Column#10, cast(test.ts.id, var_string(20))->Column#11, test.ts.col_2->Column#12", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#9, funcs:group_concat(distinct Column#6, Column#7, Column#8 separator \",\")->Column#5", + " └─Projection 8000.00 mpp[tiflash] test.ts.col_0->Column#6, test.ts.col_1->Column#7, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_2->Column#9", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.col_2, test.ts.id, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, funcs:group_concat(Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#6, test.ts.col_1->Column#7, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0->Column#9, test.ts.col_2->Column#10", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, funcs:group_concat(distinct Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 8000.00 mpp[tiflash] test.ts.col_0->Column#6, test.ts.col_1->Column#7, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0->Column#9, test.ts.col_2->Column#10", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.col_2, test.ts.id, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_1, id order by col_0) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#9, funcs:group_concat(Column#6, Column#7 order by Column#8 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_1->Column#6, cast(test.ts.id, var_string(20))->Column#7, test.ts.col_0->Column#8, test.ts.col_2->Column#9", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_1, id order by col_0) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#9, funcs:group_concat(distinct Column#6, Column#7 order by Column#8 separator \",\")->Column#5", + " └─Projection 8000.00 mpp[tiflash] test.ts.col_1->Column#6, cast(test.ts.id, var_string(20))->Column#7, test.ts.col_0->Column#8, test.ts.col_2->Column#9", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_1, test.ts.col_2, test.ts.id, funcs:firstrow(test.ts.col_0)->test.ts.col_0", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(*),min(col_0),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#20, funcs:group_concat(Column#13, Column#14, Column#15 order by Column#16 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:min(Column#17)->Column#7, funcs:count(Column#18)->Column#11, funcs:sum(Column#19)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#13, test.ts.col_1->Column#14, cast(test.ts.id, var_string(20))->Column#15, test.ts.col_0->Column#16, test.ts.col_0->Column#17, test.ts.id->Column#18, cast(test.ts.id, decimal(10,0) BINARY)->Column#19, test.ts.col_2->Column#20", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(*),max(col_1),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#19, 0), 1, Column#19), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#32, funcs:group_concat(distinct Column#25, Column#26, Column#27 order by Column#28 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:max(Column#29)->Column#7, funcs:count(Column#30)->Column#19, funcs:sum(Column#31)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#25, test.ts.col_1->Column#26, cast(test.ts.id, var_string(20))->Column#27, test.ts.col_0->Column#28, test.ts.col_1->Column#29, test.ts.id->Column#30, cast(test.ts.id, decimal(10,0) BINARY)->Column#31, test.ts.col_2->Column#32", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(distinct id),min(col_0),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#10, 0), 1, Column#10), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#11, Column#12, Column#13 order by Column#14 separator \",\")->Column#5, funcs:count(Column#15)->Column#6, funcs:min(Column#16)->Column#7, funcs:count(Column#17)->Column#10, funcs:sum(Column#18)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#11, test.ts.col_1->Column#12, cast(test.ts.id, var_string(20))->Column#13, test.ts.col_0->Column#14, test.ts.id->Column#15, test.ts.col_0->Column#16, test.ts.id->Column#17, cast(test.ts.id, decimal(10,0) BINARY)->Column#18", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(distinct id),max(col_1),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#26, Column#27, Column#28 order by Column#29 separator \",\")->Column#5, funcs:sum(Column#30)->Column#6, funcs:max(Column#31)->Column#7, funcs:sum(Column#32)->Column#14, funcs:sum(Column#33)->Column#8", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0->Column#26, test.ts.col_1->Column#27, cast(test.ts.id, var_string(20))->Column#28, test.ts.col_0->Column#29, Column#15->Column#30, Column#16->Column#31, Column#17->Column#32, Column#18->Column#33", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#23, Column#24, Column#25, funcs:count(Column#19)->Column#15, funcs:max(Column#20)->Column#16, funcs:count(Column#21)->Column#17, funcs:sum(Column#22)->Column#18", + " └─Projection 10000.00 mpp[tiflash] test.ts.id->Column#19, test.ts.col_1->Column#20, test.ts.id->Column#21, cast(test.ts.id, decimal(10,0) BINARY)->Column#22, test.ts.col_0->Column#23, test.ts.col_1->Column#24, test.ts.id->Column#25", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),min(col_0),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#21, 0), 1, Column#21), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#35, funcs:group_concat(Column#28, Column#29, Column#30 separator \",\")->Column#5, funcs:count(Column#31)->Column#6, funcs:min(Column#32)->Column#7, funcs:count(Column#33)->Column#21, funcs:sum(Column#34)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#28, test.ts.col_1->Column#29, cast(test.ts.id, var_string(20))->Column#30, test.ts.id->Column#31, test.ts.col_0->Column#32, test.ts.id->Column#33, cast(test.ts.id, decimal(10,0) BINARY)->Column#34, test.ts.col_2->Column#35", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),max(col_1),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#19, 0), 1, Column#19), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#32, funcs:group_concat(distinct Column#25, Column#26, Column#27 separator \",\")->Column#5, funcs:count(Column#28)->Column#6, funcs:max(Column#29)->Column#7, funcs:count(Column#30)->Column#19, funcs:sum(Column#31)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#25, test.ts.col_1->Column#26, cast(test.ts.id, var_string(20))->Column#27, test.ts.id->Column#28, test.ts.col_1->Column#29, test.ts.id->Column#30, cast(test.ts.id, decimal(10,0) BINARY)->Column#31, test.ts.col_2->Column#32", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),min(col_0),avg(id) from ts", + "Plan": [ + "HashAgg 1.00 root funcs:group_concat(Column#9 separator \",\")->Column#5, funcs:count(Column#10)->Column#6, funcs:min(Column#11)->Column#7, funcs:avg(Column#12, Column#13)->Column#8", + "└─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#19, Column#20, Column#21 separator \",\")->Column#9, funcs:count(Column#22)->Column#10, funcs:min(Column#23)->Column#11, funcs:count(Column#24)->Column#12, funcs:sum(Column#25)->Column#13", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#19, test.ts.col_1->Column#20, cast(test.ts.id, var_string(20))->Column#21, test.ts.id->Column#22, test.ts.col_0->Column#23, test.ts.id->Column#24, cast(test.ts.id, decimal(10,0) BINARY)->Column#25", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),max(col_1),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#26, Column#27, Column#28 separator \",\")->Column#5, funcs:sum(Column#29)->Column#6, funcs:max(Column#30)->Column#7, funcs:sum(Column#31)->Column#14, funcs:sum(Column#32)->Column#8", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0->Column#26, test.ts.col_1->Column#27, cast(test.ts.id, var_string(20))->Column#28, Column#15->Column#29, Column#16->Column#30, Column#17->Column#31, Column#18->Column#32", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#23, Column#24, Column#25, funcs:count(Column#19)->Column#15, funcs:max(Column#20)->Column#16, funcs:count(Column#21)->Column#17, funcs:sum(Column#22)->Column#18", + " └─Projection 10000.00 mpp[tiflash] test.ts.id->Column#19, test.ts.col_1->Column#20, test.ts.id->Column#21, cast(test.ts.id, decimal(10,0) BINARY)->Column#22, test.ts.col_0->Column#23, test.ts.col_1->Column#24, test.ts.id->Column#25", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),group_concat(col_0 order by 1),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#15, 0), 1, Column#15), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#27, funcs:group_concat(Column#19, Column#20, Column#21 separator \",\")->Column#5, funcs:count(Column#22)->Column#6, funcs:group_concat(Column#23 order by Column#24 separator \",\")->Column#7, funcs:count(Column#25)->Column#15, funcs:sum(Column#26)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#19, test.ts.col_1->Column#20, cast(test.ts.id, var_string(20))->Column#21, test.ts.id->Column#22, test.ts.col_0->Column#23, test.ts.col_0->Column#24, test.ts.id->Column#25, cast(test.ts.id, decimal(10,0) BINARY)->Column#26, test.ts.col_2->Column#27", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0),count(distinct id),group_concat(col_1, id order by 1,2),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#13, 0), 1, Column#13), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#24, funcs:group_concat(distinct Column#16 separator \",\")->Column#5, funcs:count(Column#17)->Column#6, funcs:group_concat(Column#18, Column#19 order by Column#20, Column#21 separator \",\")->Column#7, funcs:count(Column#22)->Column#13, funcs:sum(Column#23)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#16, test.ts.id->Column#17, test.ts.col_1->Column#18, cast(test.ts.id, var_string(20))->Column#19, test.ts.col_1->Column#20, test.ts.id->Column#21, test.ts.id->Column#22, cast(test.ts.id, decimal(10,0) BINARY)->Column#23, test.ts.col_2->Column#24", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, id),count(distinct id),group_concat(col_1, id order by 1,2),min(col_0),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#13, 0), 1, Column#13), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#16, Column#17 separator \",\")->Column#5, funcs:count(Column#18)->Column#6, funcs:group_concat(Column#19, Column#20 order by Column#21, Column#22 separator \",\")->Column#7, funcs:min(Column#23)->Column#8, funcs:count(Column#24)->Column#13, funcs:sum(Column#25)->Column#9", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#16, cast(test.ts.id, var_string(20))->Column#17, test.ts.id->Column#18, test.ts.col_1->Column#19, cast(test.ts.id, var_string(20))->Column#20, test.ts.col_1->Column#21, test.ts.id->Column#22, test.ts.col_0->Column#23, test.ts.id->Column#24, cast(test.ts.id, decimal(10,0) BINARY)->Column#25", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),group_concat(col_1, id order by 1,2),max(col_1),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#12, 0), 1, Column#12), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#14, Column#15, Column#16 separator \",\")->Column#5, funcs:count(Column#17)->Column#6, funcs:group_concat(Column#18, Column#19 order by Column#20, Column#21 separator \",\")->Column#7, funcs:max(Column#22)->Column#8, funcs:count(Column#23)->Column#12, funcs:sum(Column#24)->Column#9", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#14, test.ts.col_1->Column#15, cast(test.ts.id, var_string(20))->Column#16, test.ts.id->Column#17, test.ts.col_1->Column#18, cast(test.ts.id, var_string(20))->Column#19, test.ts.col_1->Column#20, test.ts.id->Column#21, test.ts.col_1->Column#22, test.ts.id->Column#23, cast(test.ts.id, decimal(10,0) BINARY)->Column#24", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct col_2),group_concat(col_1, id),max(col_1),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#15, 0), 1, Column#15), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#29, Column#30, Column#31 separator \",\")->Column#5, funcs:count(distinct Column#32)->Column#6, funcs:group_concat(Column#33 separator \",\")->Column#7, funcs:max(Column#34)->Column#8, funcs:sum(Column#35)->Column#15, funcs:sum(Column#36)->Column#9", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0->Column#29, test.ts.col_1->Column#30, cast(test.ts.id, var_string(20))->Column#31, test.ts.col_2->Column#32, Column#16->Column#33, Column#17->Column#34, Column#18->Column#35, Column#19->Column#36", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#25, Column#26, Column#27, Column#28, funcs:group_concat(Column#20, Column#21 separator \",\")->Column#16, funcs:max(Column#22)->Column#17, funcs:count(Column#23)->Column#18, funcs:sum(Column#24)->Column#19", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_1->Column#20, cast(test.ts.id, var_string(20))->Column#21, test.ts.col_1->Column#22, test.ts.id->Column#23, cast(test.ts.id, decimal(10,0) BINARY)->Column#24, test.ts.col_0->Column#25, test.ts.col_1->Column#26, test.ts.id->Column#27, test.ts.col_2->Column#28", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct col_2),group_concat(col_1, id),max(col_1),avg(id) from ts group by col_0", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#20, 0), 1, Column#20), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#35, funcs:group_concat(distinct Column#26, Column#27, Column#28 separator \",\")->Column#5, funcs:count(distinct Column#29)->Column#6, funcs:group_concat(Column#30, Column#31 separator \",\")->Column#7, funcs:max(Column#32)->Column#8, funcs:count(Column#33)->Column#20, funcs:sum(Column#34)->Column#9", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#26, test.ts.col_1->Column#27, cast(test.ts.id, var_string(20))->Column#28, test.ts.col_2->Column#29, test.ts.col_1->Column#30, cast(test.ts.id, var_string(20))->Column#31, test.ts.col_1->Column#32, test.ts.id->Column#33, cast(test.ts.id, decimal(10,0) BINARY)->Column#34, test.ts.col_0->Column#35", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_0, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#8, var_string(20))->Column#10, Column#9->Column#11", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", 0, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'01') from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#8, var_string(20))->Column#10, Column#9->Column#11", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:\"01\", 0, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,1) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#8, var_string(20))->Column#10, cast(Column#9, var_string(20))->Column#11", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:0, 1, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#8, Column#9 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#7, var_string(20))->Column#8, cast(Column#7, var_string(20))->Column#9", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:0, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,10) from ts group by '010'", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#17, funcs:group_concat(distinct Column#15, Column#16 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#13, var_string(20))->Column#15, cast(Column#14, var_string(20))->Column#16, Column#12->Column#17", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#12, collate: binary]", + " └─HashAgg 1.00 mpp[tiflash] group by:0, 1, 10, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts group by '011'", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#14, funcs:group_concat(distinct Column#12, Column#13 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#11, var_string(20))->Column#12, cast(Column#11, var_string(20))->Column#13, Column#10->Column#14", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#10, collate: binary]", + " └─HashAgg 1.00 mpp[tiflash] group by:0, 1, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts group by 'GG'", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#17, funcs:group_concat(distinct Column#15, Column#16 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#13, var_string(20))->Column#15, Column#14->Column#16, Column#12->Column#17", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#12, collate: binary]", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", 0, 1, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG','GG') from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#7, Column#7 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'Gg','GG') from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#8, Column#9 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", \"Gg\", ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG-10','GG') from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#8, Column#9 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", \"GG-10\", ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct '1200-01-01 00:00:00.023',1200) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] Column#8->Column#10, cast(Column#9, var_string(20))->Column#11", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:\"1200-01-01 00:00:00.023\", 1200, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", + "[types:1292]Truncated incorrect DOUBLE value: '1200-01-01 00:00:00.023'", + "[types:1292]Truncated incorrect DOUBLE value: '1200-01-01 00:00:00.023'" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0) from ts group by id", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.id, funcs:group_concat(test.ts.col_0, test.ts.col_0 separator \",\")->Column#5", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.id, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0,id) from ts group by id", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#12, funcs:group_concat(Column#9, Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#9, test.ts.col_0->Column#10, cast(test.ts.id, var_string(20))->Column#11, test.ts.id->Column#12", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.id, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct test.ts.col_0 order by Column#7 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#9, funcs:firstrow(Column#8)->Column#7", + " └─Projection 10000.00 mpp[tiflash] lt(test.ts.id, 10)->Column#8, test.ts.col_0->Column#9", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts group by col_1", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_1, funcs:group_concat(distinct test.ts.col_0 order by Column#8 separator \",\")->Column#5", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_1, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, Column#11, funcs:firstrow(Column#9)->Column#8", + " └─Projection 10000.00 mpp[tiflash] lt(test.ts.id, 10)->Column#9, test.ts.col_1->Column#10, test.ts.col_0->Column#11", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0>10 order by id<10) from ts group by col_1", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#17, funcs:group_concat(distinct Column#15 order by Column#16 separator \",\")->Column#5", + " └─Projection 8000.00 mpp[tiflash] cast(Column#10, var_string(20))->Column#15, Column#11->Column#16, test.ts.col_1->Column#17", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_1, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, Column#14, funcs:firstrow(Column#12)->Column#11", + " └─Projection 10000.00 mpp[tiflash] lt(test.ts.id, 10)->Column#12, test.ts.col_1->Column#13, gt(cast(test.ts.col_0, double BINARY), 10)->Column#14", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by col_0<=>null) from ts", + "Plan": [ + "HashAgg 1.00 root funcs:group_concat(distinct Column#6 order by Column#7 separator \",\")->Column#5", + "└─Projection 10000.00 root test.ts.col_0->Column#6, nulleq(test.ts.col_0, )->Column#7", + " └─TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", + "Scalar function 'nulleq'(signature: NullEQString, return type: bigint(1)) is not supported to push down to tiflash now.", + "Aggregation can not be pushed to tiflash because arguments of AggFunc `group_concat` contains unsupported exprs in order-by clause", + "Scalar function 'nulleq'(signature: NullEQString, return type: bigint(1)) is not supported to push down to tiflash now.", + "Aggregation can not be pushed to tiflash because arguments of AggFunc `group_concat` contains unsupported exprs in order-by clause" + ] + } + ] + }, + { + "Name": "TestIssue32632", + "Cases": [ + { + "SQL": "explain format = 'brief' select sum(ps_supplycost) from partsupp, supplier where ps_suppkey = s_suppkey;", + "Plan": [ + "HashAgg 1.00 root funcs:sum(Column#15)->Column#14", + "└─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:sum(test.partsupp.ps_supplycost)->Column#15", + " └─Projection 12500.00 mpp[tiflash] test.partsupp.ps_supplycost", + " └─HashJoin 12500.00 mpp[tiflash] inner join, equal:[eq(test.supplier.s_suppkey, test.partsupp.ps_suppkey)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:supplier keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 800000.00 mpp[tiflash] table:partsupp keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestTiFlashPartitionTableScan", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from rp_t where a = 1 or a = 20", + "Plan": [ + "TableReader 20.00 root partition:p0,p3 MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 20.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 20.00 mpp[tiflash] or(eq(test.rp_t.a, 1), eq(test.rp_t.a, 20))", + " └─TableFullScan 10000.00 mpp[tiflash] table:rp_t pushed down filter:empty, keep order:false, stats:pseudo, PartitionTableScan:true" + ] + }, + { + "SQL": "explain format = 'brief' select * from hp_t where a = 1 or a = 20", + "Plan": [ + "TableReader 20.00 root partition:p0,p1 MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 20.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 20.00 mpp[tiflash] or(eq(test.hp_t.a, 1), eq(test.hp_t.a, 20))", + " └─TableFullScan 10000.00 mpp[tiflash] table:hp_t pushed down filter:empty, keep order:false, stats:pseudo, PartitionTableScan:true" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from rp_t where a = 1 or a = 20", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#4)->Column#3", + "└─TableReader 1.00 root partition:p0,p3 MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#4", + " └─Selection 20.00 mpp[tiflash] or(eq(test.rp_t.a, 1), eq(test.rp_t.a, 20))", + " └─TableFullScan 10000.00 mpp[tiflash] table:rp_t pushed down filter:empty, keep order:false, stats:pseudo, PartitionTableScan:true" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from hp_t where a = 1 or a = 20", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#4)->Column#3", + "└─TableReader 1.00 root partition:p0,p1 MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#4", + " └─Selection 20.00 mpp[tiflash] or(eq(test.hp_t.a, 1), eq(test.hp_t.a, 20))", + " └─TableFullScan 10000.00 mpp[tiflash] table:hp_t pushed down filter:empty, keep order:false, stats:pseudo, PartitionTableScan:true" + ] + } + ] + }, + { + "Name": "TestTiFlashFineGrainedShuffle", + "Cases": [ + { + "SQL": "explain format = 'brief' select row_number() over w1 from t1 window w1 as (partition by c1 order by c1);", + "Plan": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] Column#5->Column#6, stream_count: 8", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#5 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1, rank() over w2 from t1 window w1 as (partition by c1 order by c1), w2 as (partition by c2);", + "Plan": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] Column#7->Column#8, Column#6->Column#9, stream_count: 8", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#7 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Window 10000.00 mpp[tiflash] rank()->Column#6 over(partition by test.t1.c2), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1, rank() over w2 from t1 window w1 as (partition by c1 order by c1), w2 as (partition by c2) order by 1, 2 limit 10;", + "Plan": [ + "Projection 10.00 root Column#7->Column#8, Column#6->Column#9", + "└─TopN 10.00 root Column#7, Column#6, offset:0, count:10", + " └─TableReader 10.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TopN 10.00 mpp[tiflash] Column#7, Column#6, offset:0, count:10", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#7 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Window 10000.00 mpp[tiflash] rank()->Column#6 over(partition by test.t1.c2), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1, count(c2) from t1 group by c1 having c1 > 10 window w1 as (partition by c2 order by c2);", + "Plan": [ + "TableReader 2666.67 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2666.67 mpp[tiflash] Column#6->Column#7, Column#4->Column#8, stream_count: 8", + " └─Window 2666.67 mpp[tiflash] row_number()->Column#6 over(partition by test.t1.c2 order by test.t1.c2 rows between current row and current row), stream_count: 8", + " └─Sort 2666.67 mpp[tiflash] test.t1.c2, test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 2666.67 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", + " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c2", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:sum(Column#9)->Column#4, funcs:firstrow(Column#10)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#9, funcs:firstrow(test.t1.c2)->Column#10", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, 10)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1, count(c1) from t1 group by c2 having c2 > 10 window w1 as (partition by c1 order by c2);", + "Plan": [ + "TableReader 2666.67 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2666.67 mpp[tiflash] Column#6->Column#7, Column#4->Column#8, stream_count: 8", + " └─Window 2666.67 mpp[tiflash] row_number()->Column#6 over(partition by test.t1.c1 order by test.t1.c2 rows between current row and current row), stream_count: 8", + " └─Sort 2666.67 mpp[tiflash] test.t1.c1, test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 2666.67 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c1, test.t1.c2", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:sum(Column#9)->Column#4, funcs:firstrow(Column#10)->test.t1.c1, funcs:firstrow(test.t1.c2)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:count(test.t1.c1)->Column#9, funcs:firstrow(test.t1.c1)->Column#10", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c2, 10)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1 from t1 a join t1 b on a.c1 = b.c2 window w1 as (partition by a.c1);", + "Plan": [ + "TableReader 12487.50 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 12487.50 mpp[tiflash] Column#8->Column#9, stream_count: 8", + " └─Window 12487.50 mpp[tiflash] row_number()->Column#8 over(partition by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 12487.50 mpp[tiflash] test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 12487.50 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t1.c1, test.t1.c2)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.c1))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:a pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t1.c2))", + " └─TableFullScan 10000.00 mpp[tiflash] table:b pushed down filter:empty, keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1 from t1 where c1 < 100 window w1 as (partition by c1 order by c1);", + "Plan": [ + "TableReader 3323.33 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 3323.33 mpp[tiflash] Column#5->Column#6, stream_count: 8", + " └─Window 3323.33 mpp[tiflash] row_number()->Column#5 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 3323.33 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 3323.33 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Selection 3323.33 mpp[tiflash] lt(test.t1.c1, 100)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t1;", + "Plan": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1 from t1 window w1 as (order by c1);", + "Plan": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] Column#5->Column#6", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#5 over(order by test.t1.c1 rows between current row and current row)", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1, count(c2) from t1 group by c1 having c1 > 10 window w1 as (partition by c1 order by c2);", + "Plan": [ + "TableReader 2666.67 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2666.67 mpp[tiflash] Column#6->Column#7, Column#4->Column#8", + " └─Window 2666.67 mpp[tiflash] row_number()->Column#6 over(partition by test.t1.c1 order by test.t1.c2 rows between current row and current row)", + " └─Sort 2666.67 mpp[tiflash] test.t1.c1, test.t1.c2", + " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c1, test.t1.c2", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:sum(Column#9)->Column#4, funcs:firstrow(test.t1.c1)->test.t1.c1, funcs:firstrow(Column#11)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#9, funcs:firstrow(test.t1.c2)->Column#11", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, 10)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ] + } + ] + } +] diff --git a/planner/cascades/optimize.go b/planner/cascades/optimize.go index 29be0272e011e..5a5f6fa6e8f20 100644 --- a/planner/cascades/optimize.go +++ b/planner/cascades/optimize.go @@ -115,8 +115,14 @@ func (opt *Optimizer) FindBestPlan(sctx sessionctx.Context, logical plannercore. return p, cost, err } +<<<<<<< HEAD:planner/cascades/optimize.go func (*Optimizer) onPhasePreprocessing(_ sessionctx.Context, plan plannercore.LogicalPlan) (plannercore.LogicalPlan, error) { err := plan.PruneColumns(plan.Schema().Columns, nil) +======= +func (*Optimizer) onPhasePreprocessing(_ plannercore.PlanContext, plan plannercore.LogicalPlan) (plannercore.LogicalPlan, error) { + var err error + plan, err = plan.PruneColumns(plan.Schema().Columns, nil) +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/cascades/optimize.go if err != nil { return nil, err } diff --git a/planner/core/plan.go b/planner/core/plan.go index 373c1d7162c4e..5a32104f49111 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -257,8 +257,13 @@ type LogicalPlan interface { // Because it might change the root if the having clause exists, we need to return a plan that represents a new root. PredicatePushDown([]expression.Expression, *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) +<<<<<<< HEAD:planner/core/plan.go // PruneColumns prunes the unused columns. PruneColumns([]*expression.Column, *logicalOptimizeOp) error +======= + // PruneColumns prunes the unused columns, and return the new logical plan if changed, otherwise it's same. + PruneColumns([]*expression.Column, *logicalOptimizeOp) (LogicalPlan, error) +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/plan.go // findBestTask converts the logical plan to the physical plan. It's a new interface. // It is called recursively from the parent to the children to create the result physical plan. @@ -753,10 +758,15 @@ func (*baseLogicalPlan) ExtractCorrelatedCols() []*expression.CorrelatedColumn { } // PruneColumns implements LogicalPlan interface. +<<<<<<< HEAD:planner/core/plan.go func (p *baseLogicalPlan) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { +======= +func (p *baseLogicalPlan) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/plan.go if len(p.children) == 0 { - return nil + return p.self, nil } +<<<<<<< HEAD:planner/core/plan.go return p.children[0].PruneColumns(parentUsedCols, opt) } @@ -828,6 +838,14 @@ func (p *basePlan) MemoryUsage() (sum int64) { sum = basePlanSize + int64(len(p.tp)) return sum +======= + var err error + p.children[0], err = p.children[0].PruneColumns(parentUsedCols, opt) + if err != nil { + return nil, err + } + return p.self, nil +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/plan.go } // Schema implements Plan Schema interface. diff --git a/planner/core/resolve_indices.go b/planner/core/resolve_indices.go index a9e1613faf6f0..c9ae1dc42fce4 100644 --- a/planner/core/resolve_indices.go +++ b/planner/core/resolve_indices.go @@ -141,6 +141,7 @@ func (p *PhysicalHashJoin) ResolveIndicesItself() (err error) { copy(shallowColSlice, p.schema.Columns) p.schema = expression.NewSchema(shallowColSlice...) foundCnt := 0 +<<<<<<< HEAD:planner/core/resolve_indices.go // The two column sets are all ordered. And the colsNeedResolving is the subset of the mergedSchema. // So we can just move forward j if there's no matching is found. // We don't use the normal ResolvIndices here since there might be duplicate columns in the schema. @@ -150,17 +151,35 @@ func (p *PhysicalHashJoin) ResolveIndicesItself() (err error) { if !p.schema.Columns[i].Equal(nil, mergedSchema.Columns[j]) { j++ continue +======= + + // Here we want to resolve all join schema columns directly as a merged schema, and you know same name + // col in join schema should be separately redirected to corresponded same col in child schema. But two + // column sets are **NOT** always ordered, see comment: https://github.com/pingcap/tidb/pull/45831#discussion_r1481031471 + // we are using mapping mechanism instead of moving j forward. + marked := make([]bool, mergedSchema.Len()) + for i := 0; i < colsNeedResolving; i++ { + findIdx := -1 + for j := 0; j < len(mergedSchema.Columns); j++ { + if !p.schema.Columns[i].EqualColumn(mergedSchema.Columns[j]) || marked[j] { + continue + } + // resolve to a same unique id one, and it not being marked. + findIdx = j + break + } + if findIdx != -1 { + // valid one. + p.schema.Columns[i] = p.schema.Columns[i].Clone().(*expression.Column) + p.schema.Columns[i].Index = findIdx + marked[findIdx] = true + foundCnt++ +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/resolve_indices.go } - p.schema.Columns[i] = p.schema.Columns[i].Clone().(*expression.Column) - p.schema.Columns[i].Index = j - i++ - j++ - foundCnt++ } if foundCnt < colsNeedResolving { return errors.Errorf("Some columns of %v cannot find the reference from its child(ren)", p.ExplainID().String()) } - return } diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index ece127327414b..cb0f96e2a809b 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -31,9 +31,19 @@ import ( type columnPruner struct { } +<<<<<<< HEAD:planner/core/rule_column_pruning.go func (*columnPruner) optimize(_ context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { err := lp.PruneColumns(lp.Schema().Columns, opt) return lp, err +======= +func (*columnPruner) optimize(_ context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, bool, error) { + planChanged := false + lp, err := lp.PruneColumns(lp.Schema().Columns, opt) + if err != nil { + return nil, planChanged, err + } + return lp, planChanged, nil +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go } // ExprsHasSideEffects checks if any of the expressions has side effects. @@ -63,11 +73,47 @@ func exprHasSetVarOrSleep(expr expression.Expression) bool { return false } +<<<<<<< HEAD:planner/core/rule_column_pruning.go // PruneColumns implements LogicalPlan interface. // If any expression has SetVar function or Sleep function, we do not prune it. func (p *LogicalProjection) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { child := p.children[0] used := expression.GetUsedList(parentUsedCols, p.schema) +======= +// PruneColumns implement the Expand OP's column pruning logic. +// logicExpand is built in the logical plan building phase, where all the column prune is not done yet. So the +// expand projection expressions is meaningless if it built at that time. (we only maintain its schema, while +// the level projection expressions construction is left to the last logical optimize rule) +// +// so when do the rule_column_pruning here, we just prune the schema is enough. +func (p *LogicalExpand) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { + // Expand need those extra redundant distinct group by columns projected from underlying projection. + // distinct GroupByCol must be used by aggregate above, to make sure this, append distinctGroupByCol again. + parentUsedCols = append(parentUsedCols, p.distinctGroupByCol...) + used := expression.GetUsedList(p.SCtx(), parentUsedCols, p.Schema()) + prunedColumns := make([]*expression.Column, 0) + for i := len(used) - 1; i >= 0; i-- { + if !used[i] { + prunedColumns = append(prunedColumns, p.schema.Columns[i]) + p.schema.Columns = append(p.schema.Columns[:i], p.schema.Columns[i+1:]...) + p.names = append(p.names[:i], p.names[i+1:]...) + } + } + appendColumnPruneTraceStep(p, prunedColumns, opt) + // Underlying still need to keep the distinct group by columns and parent used columns. + var err error + p.children[0], err = p.children[0].PruneColumns(parentUsedCols, opt) + if err != nil { + return nil, err + } + return p, nil +} + +// PruneColumns implements LogicalPlan interface. +// If any expression has SetVar function or Sleep function, we do not prune it. +func (p *LogicalProjection) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { + used := expression.GetUsedList(p.SCtx(), parentUsedCols, p.schema) +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go prunedColumns := make([]*expression.Column, 0) for i := len(used) - 1; i >= 0; i-- { @@ -80,6 +126,7 @@ func (p *LogicalProjection) PruneColumns(parentUsedCols []*expression.Column, op appendColumnPruneTraceStep(p, prunedColumns, opt) selfUsedCols := make([]*expression.Column, 0, len(p.Exprs)) selfUsedCols = expression.ExtractColumnsFromExpressions(selfUsedCols, p.Exprs, nil) +<<<<<<< HEAD:planner/core/rule_column_pruning.go return child.PruneColumns(selfUsedCols, opt) } @@ -92,6 +139,30 @@ func (p *LogicalSelection) PruneColumns(parentUsedCols []*expression.Column, opt // PruneColumns implements LogicalPlan interface. func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { +======= + var err error + p.children[0], err = p.children[0].PruneColumns(selfUsedCols, opt) + if err != nil { + return nil, err + } + return p, nil +} + +// PruneColumns implements LogicalPlan interface. +func (p *LogicalSelection) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { + child := p.children[0] + parentUsedCols = expression.ExtractColumnsFromExpressions(parentUsedCols, p.Conditions, nil) + var err error + p.children[0], err = child.PruneColumns(parentUsedCols, opt) + if err != nil { + return nil, err + } + return p, nil +} + +// PruneColumns implements LogicalPlan interface. +func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go child := la.children[0] used := expression.GetUsedList(parentUsedCols, la.Schema()) prunedColumns := make([]*expression.Column, 0) @@ -137,7 +208,7 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column, newAgg, err = aggregation.NewAggFuncDesc(la.ctx, ast.AggFuncCount, []expression.Expression{expression.NewOne()}, false) } if err != nil { - return err + return nil, err } la.AggFuncs = append(la.AggFuncs, newAgg) col := &expression.Column{ @@ -164,10 +235,17 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column, } } appendGroupByItemsPruneTraceStep(la, prunedGroupByItems, opt) +<<<<<<< HEAD:planner/core/rule_column_pruning.go err := child.PruneColumns(selfUsedCols, opt) +======= + var err error + la.children[0], err = child.PruneColumns(selfUsedCols, opt) +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go if err != nil { - return err + return nil, err } + // update children[0] + child = la.children[0] // Do an extra Projection Elimination here. This is specially for empty Projection below Aggregation. // This kind of Projection would cause some bugs for MPP plan and is safe to be removed. // This kind of Projection should be removed in Projection Elimination, but currently PrunColumnsAgain is @@ -178,7 +256,7 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column, la.SetChildren(childOfChild) } } - return nil + return la, nil } func pruneByItems(p LogicalPlan, old []*util.ByItems, opt *logicalOptimizeOp) (byItems []*util.ByItems, @@ -215,28 +293,59 @@ func pruneByItems(p LogicalPlan, old []*util.ByItems, opt *logicalOptimizeOp) (b // 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(). +<<<<<<< HEAD:planner/core/rule_column_pruning.go func (ls *LogicalSort) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { child := ls.children[0] var cols []*expression.Column ls.ByItems, cols = pruneByItems(ls, ls.ByItems, opt) parentUsedCols = append(parentUsedCols, cols...) return child.PruneColumns(parentUsedCols, opt) +======= +func (ls *LogicalSort) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { + var cols []*expression.Column + ls.ByItems, cols = pruneByItems(ls, ls.ByItems, opt) + parentUsedCols = append(parentUsedCols, cols...) + var err error + ls.children[0], err = ls.children[0].PruneColumns(parentUsedCols, opt) + if err != nil { + return nil, err + } + return ls, nil +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go } // 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(). +<<<<<<< HEAD:planner/core/rule_column_pruning.go func (lt *LogicalTopN) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { +======= +func (lt *LogicalTopN) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go child := lt.children[0] var cols []*expression.Column lt.ByItems, cols = pruneByItems(lt, lt.ByItems, opt) parentUsedCols = append(parentUsedCols, cols...) +<<<<<<< HEAD:planner/core/rule_column_pruning.go return child.PruneColumns(parentUsedCols, opt) } // PruneColumns implements LogicalPlan interface. func (p *LogicalUnionAll) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { used := expression.GetUsedList(parentUsedCols, p.schema) +======= + var err error + lt.children[0], err = child.PruneColumns(parentUsedCols, opt) + if err != nil { + return nil, err + } + return lt, nil +} + +// PruneColumns implements LogicalPlan interface. +func (p *LogicalUnionAll) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { + used := expression.GetUsedList(p.SCtx(), parentUsedCols, p.schema) +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go hasBeenUsed := false for i := range used { hasBeenUsed = hasBeenUsed || used[i] @@ -251,10 +360,16 @@ func (p *LogicalUnionAll) PruneColumns(parentUsedCols []*expression.Column, opt used[i] = true } } +<<<<<<< HEAD:planner/core/rule_column_pruning.go for _, child := range p.Children() { err := child.PruneColumns(parentUsedCols, opt) +======= + var err error + for i, child := range p.Children() { + p.Children()[i], err = child.PruneColumns(parentUsedCols, opt) +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go if err != nil { - return err + return nil, err } } @@ -286,11 +401,15 @@ func (p *LogicalUnionAll) PruneColumns(parentUsedCols []*expression.Column, opt } } } - return nil + return p, nil } // PruneColumns implements LogicalPlan interface. +<<<<<<< HEAD:planner/core/rule_column_pruning.go func (p *LogicalUnionScan) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { +======= +func (p *LogicalUnionScan) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go for i := 0; i < p.handleCols.NumCols(); i++ { parentUsedCols = append(parentUsedCols, p.handleCols.GetCol(i)) } @@ -301,12 +420,26 @@ func (p *LogicalUnionScan) PruneColumns(parentUsedCols []*expression.Column, opt } condCols := expression.ExtractColumnsFromExpressions(nil, p.conditions, nil) parentUsedCols = append(parentUsedCols, condCols...) +<<<<<<< HEAD:planner/core/rule_column_pruning.go return p.children[0].PruneColumns(parentUsedCols, opt) } // PruneColumns implements LogicalPlan interface. func (ds *DataSource) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { used := expression.GetUsedList(parentUsedCols, ds.schema) +======= + var err error + p.children[0], err = p.children[0].PruneColumns(parentUsedCols, opt) + if err != nil { + return nil, err + } + return p, nil +} + +// PruneColumns implements LogicalPlan interface. +func (ds *DataSource) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { + used := expression.GetUsedList(ds.SCtx(), parentUsedCols, ds.schema) +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go exprCols := expression.ExtractColumnsFromExpressions(nil, ds.allConds, nil) exprUsed := expression.GetUsedList(exprCols, ds.schema) @@ -353,11 +486,15 @@ func (ds *DataSource) PruneColumns(parentUsedCols []*expression.Column, opt *log if ds.handleCols != nil && ds.handleCols.IsInt() && ds.schema.ColumnIndex(ds.handleCols.GetCol(0)) == -1 { ds.handleCols = nil } - return nil + return ds, nil } // PruneColumns implements LogicalPlan interface. +<<<<<<< HEAD:planner/core/rule_column_pruning.go func (p *LogicalMemTable) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { +======= +func (p *LogicalMemTable) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go switch p.TableInfo.Name.O { case infoschema.TableStatementsSummary, infoschema.TableStatementsSummaryHistory, @@ -371,7 +508,7 @@ func (p *LogicalMemTable) PruneColumns(parentUsedCols []*expression.Column, opt infoschema.TableDeadlocks, infoschema.ClusterTableDeadlocks: default: - return nil + return p, nil } prunedColumns := make([]*expression.Column, 0) used := expression.GetUsedList(parentUsedCols, p.schema) @@ -384,12 +521,17 @@ func (p *LogicalMemTable) PruneColumns(parentUsedCols []*expression.Column, opt } } appendColumnPruneTraceStep(p, prunedColumns, opt) - return nil + return p, nil } // PruneColumns implements LogicalPlan interface. +<<<<<<< HEAD:planner/core/rule_column_pruning.go func (p *LogicalTableDual) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { used := expression.GetUsedList(parentUsedCols, p.Schema()) +======= +func (p *LogicalTableDual) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { + used := expression.GetUsedList(p.SCtx(), parentUsedCols, p.Schema()) +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go prunedColumns := make([]*expression.Column, 0) for i := len(used) - 1; i >= 0; i-- { if !used[i] { @@ -398,7 +540,7 @@ func (p *LogicalTableDual) PruneColumns(parentUsedCols []*expression.Column, opt } } appendColumnPruneTraceStep(p, prunedColumns, opt) - return nil + return p, nil } func (p *LogicalJoin) extractUsedCols(parentUsedCols []*expression.Column) (leftCols []*expression.Column, rightCols []*expression.Column) { @@ -434,18 +576,30 @@ func (p *LogicalJoin) mergeSchema() { } // PruneColumns implements LogicalPlan interface. +<<<<<<< HEAD:planner/core/rule_column_pruning.go func (p *LogicalJoin) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { leftCols, rightCols := p.extractUsedCols(parentUsedCols) err := p.children[0].PruneColumns(leftCols, opt) +======= +func (p *LogicalJoin) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { + leftCols, rightCols := p.extractUsedCols(parentUsedCols) + + var err error + p.children[0], err = p.children[0].PruneColumns(leftCols, opt) +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go if err != nil { - return err + return nil, err } addConstOneForEmptyProjection(p.children[0]) +<<<<<<< HEAD:planner/core/rule_column_pruning.go err = p.children[1].PruneColumns(rightCols, opt) +======= + p.children[1], err = p.children[1].PruneColumns(rightCols, opt) +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go if err != nil { - return err + return nil, err } addConstOneForEmptyProjection(p.children[1]) @@ -455,16 +609,32 @@ func (p *LogicalJoin) PruneColumns(parentUsedCols []*expression.Column, opt *log parentUsedCols = append(parentUsedCols, joinCol) } p.inlineProjection(parentUsedCols, opt) - return nil + return p, nil } // PruneColumns implements LogicalPlan interface. +<<<<<<< HEAD:planner/core/rule_column_pruning.go func (la *LogicalApply) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { leftCols, rightCols := la.extractUsedCols(parentUsedCols) err := la.children[1].PruneColumns(rightCols, opt) +======= +func (la *LogicalApply) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { + leftCols, rightCols := la.extractUsedCols(parentUsedCols) + allowEliminateApply := fixcontrol.GetBoolWithDefault(la.SCtx().GetSessionVars().GetOptimizerFixControlMap(), fixcontrol.Fix45822, true) + var err error + if allowEliminateApply && rightCols == nil && la.JoinType == LeftOuterJoin { + applyEliminateTraceStep(la.Children()[1], opt) + resultPlan := la.Children()[0] + // reEnter the new child's column pruning, returning child[0] as a new child here. + return resultPlan.PruneColumns(parentUsedCols, opt) + } + + // column pruning for child-1. + la.children[1], err = la.children[1].PruneColumns(rightCols, opt) +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go if err != nil { - return err + return nil, err } addConstOneForEmptyProjection(la.children[1]) @@ -473,20 +643,39 @@ func (la *LogicalApply) PruneColumns(parentUsedCols []*expression.Column, opt *l leftCols = append(leftCols, &col.Column) } +<<<<<<< HEAD:planner/core/rule_column_pruning.go err = la.children[0].PruneColumns(leftCols, opt) +======= + // column pruning for child-0. + la.children[0], err = la.children[0].PruneColumns(leftCols, opt) +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go if err != nil { - return err + return nil, err } addConstOneForEmptyProjection(la.children[0]) - la.mergeSchema() - return nil + return la, nil } // PruneColumns implements LogicalPlan interface. +<<<<<<< HEAD:planner/core/rule_column_pruning.go func (p *LogicalLock) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { if !IsSelectForUpdateLockType(p.Lock.LockType) { return p.baseLogicalPlan.PruneColumns(parentUsedCols, opt) +======= +func (p *LogicalLock) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { + var err error + if !IsSelectForUpdateLockType(p.Lock.LockType) { + // when use .baseLogicalPlan to call the PruneColumns, it means current plan itself has + // nothing to pruning or plan change, so they resort to its children's column pruning logic. + // so for the returned logical plan here, p is definitely determined, we just need to collect + // those extra deeper call error in handling children's column pruning. + _, err = p.baseLogicalPlan.PruneColumns(parentUsedCols, opt) + if err != nil { + return nil, err + } + return p, nil +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go } for tblID, cols := range p.tblID2Handle { @@ -500,11 +689,23 @@ func (p *LogicalLock) PruneColumns(parentUsedCols []*expression.Column, opt *log parentUsedCols = append(parentUsedCols, physTblIDCol) } } +<<<<<<< HEAD:planner/core/rule_column_pruning.go return p.children[0].PruneColumns(parentUsedCols, opt) } // PruneColumns implements LogicalPlan interface. func (p *LogicalWindow) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { +======= + p.children[0], err = p.children[0].PruneColumns(parentUsedCols, opt) + if err != nil { + return nil, err + } + return p, nil +} + +// PruneColumns implements LogicalPlan interface. +func (p *LogicalWindow) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go windowColumns := p.GetWindowResultColumns() cnt := 0 for _, col := range parentUsedCols { @@ -522,14 +723,19 @@ func (p *LogicalWindow) PruneColumns(parentUsedCols []*expression.Column, opt *l } parentUsedCols = parentUsedCols[:cnt] parentUsedCols = p.extractUsedCols(parentUsedCols) +<<<<<<< HEAD:planner/core/rule_column_pruning.go err := p.children[0].PruneColumns(parentUsedCols, opt) +======= + var err error + p.children[0], err = p.children[0].PruneColumns(parentUsedCols, opt) +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go if err != nil { - return err + return nil, err } p.SetSchema(p.children[0].Schema().Clone()) p.Schema().Append(windowColumns...) - return nil + return p, nil } func (p *LogicalWindow) extractUsedCols(parentUsedCols []*expression.Column) []*expression.Column { @@ -548,19 +754,29 @@ func (p *LogicalWindow) extractUsedCols(parentUsedCols []*expression.Column) []* } // PruneColumns implements LogicalPlan interface. +<<<<<<< HEAD:planner/core/rule_column_pruning.go func (p *LogicalLimit) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { +======= +func (p *LogicalLimit) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go if len(parentUsedCols) == 0 { // happens when LIMIT appears in UPDATE. - return nil + return p, nil } savedUsedCols := make([]*expression.Column, len(parentUsedCols)) copy(savedUsedCols, parentUsedCols) +<<<<<<< HEAD:planner/core/rule_column_pruning.go if err := p.children[0].PruneColumns(parentUsedCols, opt); err != nil { return err +======= + var err error + if p.children[0], err = p.children[0].PruneColumns(parentUsedCols, opt); err != nil { + return nil, err +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go } p.schema = nil p.inlineProjection(savedUsedCols, opt) - return nil + return p, nil } func (*columnPruner) name() string { @@ -678,3 +894,34 @@ func preferKeyColumnFromTable(dataSource *DataSource, originColumns []*expressio } return resultColumn, resultColumnInfo } +<<<<<<< HEAD:planner/core/rule_column_pruning.go +======= + +// PruneColumns implements the interface of LogicalPlan. +// LogicalCTE just do a empty function call. It's logical optimize is indivisual phase. +func (p *LogicalCTE) PruneColumns(_ []*expression.Column, _ *logicalOptimizeOp) (LogicalPlan, error) { + return p, nil +} + +// PruneColumns implements the interface of LogicalPlan. +func (p *LogicalSequence) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { + var err error + p.children[len(p.children)-1], err = p.children[len(p.children)-1].PruneColumns(parentUsedCols, opt) + if err != nil { + return nil, err + } + return p, nil +} + +func applyEliminateTraceStep(lp LogicalPlan, opt *logicalOptimizeOp) { + action := func() string { + buffer := bytes.NewBufferString( + fmt.Sprintf("%v_%v is eliminated.", lp.TP(), lp.ID())) + return buffer.String() + } + reason := func() string { + return fmt.Sprintf("%v_%v can be eliminated because it hasn't been used by it's parent.", lp.TP(), lp.ID()) + } + opt.appendStepToCurrent(lp.ID(), lp.TP(), reason, action) +} +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_column_pruning.go diff --git a/planner/core/rule_max_min_eliminate.go b/planner/core/rule_max_min_eliminate.go index d9bb1eae19f9b..1021f22f70afb 100644 --- a/planner/core/rule_max_min_eliminate.go +++ b/planner/core/rule_max_min_eliminate.go @@ -154,9 +154,19 @@ func (a *maxMinEliminator) splitAggFuncAndCheckIndices(agg *LogicalAggregation, newAgg := LogicalAggregation{AggFuncs: []*aggregation.AggFuncDesc{f}}.Init(agg.ctx, agg.blockOffset) newAgg.SetChildren(a.cloneSubPlans(agg.children[0])) newAgg.schema = expression.NewSchema(agg.schema.Columns[i]) +<<<<<<< HEAD:planner/core/rule_max_min_eliminate.go if err := newAgg.PruneColumns([]*expression.Column{newAgg.schema.Columns[0]}, opt); err != nil { +======= + // Since LogicalAggregation doesn’t use the parent LogicalPlan, passing an incorrect parameter here won’t affect subsequent optimizations. + var ( + p LogicalPlan + err error + ) + if p, err = newAgg.PruneColumns([]*expression.Column{newAgg.schema.Columns[0]}, opt); err != nil { +>>>>>>> 58e5284b3f4 (planner,executor: fix join resolveIndex won't find its column from children schema & amend join's lused and rused logic for reversed column ref from join schema to its children (#51203)):pkg/planner/core/rule_max_min_eliminate.go return nil, false } + newAgg = p.(*LogicalAggregation) aggs = append(aggs, newAgg) } return aggs, true diff --git a/tests/integrationtest/r/planner/core/casetest/integration.result b/tests/integrationtest/r/planner/core/casetest/integration.result new file mode 100644 index 0000000000000..8fe1408485901 --- /dev/null +++ b/tests/integrationtest/r/planner/core/casetest/integration.result @@ -0,0 +1,1783 @@ +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int); +insert into t values(1),(2); +select count(1) from t join (select count(1) from t where false) as tmp; +count(1) +2 +select count(1) from t join (select max(a) from t where false) as tmp; +count(1) +2 +select count(1) from t join (select min(a) from t where false) as tmp; +count(1) +2 +select count(1) from t join (select sum(a) from t where false) as tmp; +count(1) +2 +select count(1) from t join (select avg(a) from t where false) as tmp; +count(1) +2 +select count(1) from t join (select count(1) from t where false group by a) as tmp; +count(1) +0 +select count(1) from t join (select max(a) from t where false group by a) as tmp; +count(1) +0 +select count(1) from t join (select min(a) from t where false group by a) as tmp; +count(1) +0 +select count(1) from t join (select sum(a) from t where false group by a) as tmp; +count(1) +0 +select count(1) from t join (select avg(a) from t where false group by a) as tmp; +count(1) +0 +SELECT avg(2) FROM(SELECT min(c) FROM t JOIN(SELECT 1 c) d ORDER BY a) e; +avg(2) +2.0000 +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a bigint, b bigint); +explain format = 'brief' select * from t t1 left join t t2 on t1.a=t2.a where from_unixtime(t2.b); +id estRows task access object operator info +Projection 9990.00 root planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.b, planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.b +└─HashJoin 9990.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] + ├─Selection(Build) 7992.00 root from_unixtime(cast(planner__core__casetest__integration.t.b, decimal(20,0) BINARY)) + │ └─TableReader 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int not null, b datetime default null); +explain format = 'brief' select * from t t1 left join t t2 on t1.a = t2.a where cast(t1.b as date) >= '2019-01-01'; +id estRows task access object operator info +HashJoin 10000.00 root left outer join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +├─TableReader(Build) 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] ge(cast(planner__core__casetest__integration.t.b, date BINARY), 2019-01-01 00:00:00.000000) +│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +set @@tidb_partition_prune_mode='static'; +set tidb_opt_limit_push_down_threshold=0; +drop table if exists t; +create table t(a int, b int)partition by range columns(a)(partition p0 values less than (10), partition p1 values less than(20), partition p2 values less than(30)); +insert into t values(21, 1), (22, 2), (23, 3), (24, 4), (15, 5); +analyze table t; +explain format = 'brief' select * from t order by a; +id estRows task access object operator info +Sort 10005.00 root planner__core__casetest__integration.t.a +└─PartitionUnion 10005.00 root + ├─TableReader 10000.00 root data:TableFullScan + │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo + ├─TableReader 1.00 root data:TableFullScan + │ └─TableFullScan 1.00 cop[tikv] table:t, partition:p1 keep order:false + └─TableReader 4.00 root data:TableFullScan + └─TableFullScan 4.00 cop[tikv] table:t, partition:p2 keep order:false +select * from t order by a; +a b +15 5 +21 1 +22 2 +23 3 +24 4 +explain format = 'brief' select * from t order by a limit 3; +id estRows task access object operator info +TopN 3.00 root planner__core__casetest__integration.t.a, offset:0, count:3 +└─PartitionUnion 7.00 root + ├─TopN 3.00 root planner__core__casetest__integration.t.a, offset:0, count:3 + │ └─TableReader 3.00 root data:TopN + │ └─TopN 3.00 cop[tikv] planner__core__casetest__integration.t.a, offset:0, count:3 + │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo + ├─TopN 1.00 root planner__core__casetest__integration.t.a, offset:0, count:3 + │ └─TableReader 1.00 root data:TableFullScan + │ └─TableFullScan 1.00 cop[tikv] table:t, partition:p1 keep order:false + └─TopN 3.00 root planner__core__casetest__integration.t.a, offset:0, count:3 + └─TableReader 3.00 root data:TopN + └─TopN 3.00 cop[tikv] planner__core__casetest__integration.t.a, offset:0, count:3 + └─TableFullScan 4.00 cop[tikv] table:t, partition:p2 keep order:false +select * from t order by a limit 3; +a b +15 5 +21 1 +22 2 +set tidb_opt_limit_push_down_threshold=default; +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int primary key); +set tidb_enable_clustered_index='ON'; +create table cluster_index_t(a int, b int, c int, primary key (a, b)); +explain format = 'brief' (select max(a) from t) union (select min(a) from t); +id estRows task access object operator info +HashAgg 2.00 root group by:Column#5, funcs:firstrow(Column#5)->Column#5 +└─Union 2.00 root + ├─StreamAgg 1.00 root funcs:max(planner__core__casetest__integration.t.a)->Column#2 + │ └─Limit 1.00 root offset:0, count:1 + │ └─TableReader 1.00 root data:Limit + │ └─Limit 1.00 cop[tikv] offset:0, count:1 + │ └─TableFullScan 1.00 cop[tikv] table:t keep order:true, desc, stats:pseudo + └─StreamAgg 1.00 root funcs:min(planner__core__casetest__integration.t.a)->Column#4 + └─Limit 1.00 root offset:0, count:1 + └─TableReader 1.00 root data:Limit + └─Limit 1.00 cop[tikv] offset:0, count:1 + └─TableFullScan 1.00 cop[tikv] table:t keep order:true, stats:pseudo +explain format = 'brief' select min(a), max(a) from cluster_index_t; +id estRows task access object operator info +HashJoin 1.00 root CARTESIAN inner join +├─StreamAgg(Build) 1.00 root funcs:max(planner__core__casetest__integration.cluster_index_t.a)->Column#5 +│ └─Limit 1.00 root offset:0, count:1 +│ └─TableReader 1.00 root data:Limit +│ └─Limit 1.00 cop[tikv] offset:0, count:1 +│ └─TableFullScan 1.00 cop[tikv] table:cluster_index_t keep order:true, desc, stats:pseudo +└─StreamAgg(Probe) 1.00 root funcs:min(planner__core__casetest__integration.cluster_index_t.a)->Column#4 + └─Limit 1.00 root offset:0, count:1 + └─TableReader 1.00 root data:Limit + └─Limit 1.00 cop[tikv] offset:0, count:1 + └─TableFullScan 1.00 cop[tikv] table:cluster_index_t keep order:true, stats:pseudo +explain format = 'brief' select min(b), max(b) from cluster_index_t where a = 1; +id estRows task access object operator info +HashJoin 1.00 root CARTESIAN inner join +├─StreamAgg(Build) 1.00 root funcs:max(planner__core__casetest__integration.cluster_index_t.b)->Column#5 +│ └─Limit 1.00 root offset:0, count:1 +│ └─TableReader 1.00 root data:Limit +│ └─Limit 1.00 cop[tikv] offset:0, count:1 +│ └─TableRangeScan 1.00 cop[tikv] table:cluster_index_t range:[1,1], keep order:true, desc, stats:pseudo +└─StreamAgg(Probe) 1.00 root funcs:min(planner__core__casetest__integration.cluster_index_t.b)->Column#4 + └─Limit 1.00 root offset:0, count:1 + └─TableReader 1.00 root data:Limit + └─Limit 1.00 cop[tikv] offset:0, count:1 + └─TableRangeScan 1.00 cop[tikv] table:cluster_index_t range:[1,1], keep order:true, stats:pseudo +explain format = 'brief' select min(a), max(a) from cluster_index_t where b = 1; +id estRows task access object operator info +StreamAgg 1.00 root funcs:min(Column#8)->Column#4, funcs:max(Column#9)->Column#5 +└─TableReader 1.00 root data:StreamAgg + └─StreamAgg 1.00 cop[tikv] funcs:min(planner__core__casetest__integration.cluster_index_t.a)->Column#8, funcs:max(planner__core__casetest__integration.cluster_index_t.a)->Column#9 + └─Selection 10.00 cop[tikv] eq(planner__core__casetest__integration.cluster_index_t.b, 1) + └─TableFullScan 10000.00 cop[tikv] table:cluster_index_t keep order:false, stats:pseudo +explain format = 'brief' select min(b), max(b) from cluster_index_t where b = 1; +id estRows task access object operator info +StreamAgg 1.00 root funcs:min(Column#8)->Column#4, funcs:max(Column#9)->Column#5 +└─TableReader 1.00 root data:StreamAgg + └─StreamAgg 1.00 cop[tikv] funcs:min(planner__core__casetest__integration.cluster_index_t.b)->Column#8, funcs:max(planner__core__casetest__integration.cluster_index_t.b)->Column#9 + └─Selection 10.00 cop[tikv] eq(planner__core__casetest__integration.cluster_index_t.b, 1) + └─TableFullScan 10000.00 cop[tikv] table:cluster_index_t keep order:false, stats:pseudo +set tidb_enable_clustered_index=DEFAULT; +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int, b int); +desc format = 'brief' select t1.b from t t1 where t1.b in (select t2.a from t t2 order by t1.a+t2.a limit 1); +id estRows task access object operator info +Projection 9990.00 root planner__core__casetest__integration.t.b +└─Apply 9990.00 root semi join, equal:[eq(planner__core__casetest__integration.t.b, planner__core__casetest__integration.t.a)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.b)) + │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo + └─Selection(Probe) 7992.00 root not(isnull(planner__core__casetest__integration.t.a)) + └─Projection 9990.00 root planner__core__casetest__integration.t.a + └─TopN 9990.00 root Column#7, offset:0, count:1 + └─Projection 9990.00 root planner__core__casetest__integration.t.a, plus(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)->Column#7 + └─TableReader 9990.00 root data:TopN + └─TopN 9990.00 cop[tikv] plus(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a), offset:0, count:1 + └─TableFullScan 99900000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +desc format = 'brief' select t1.a from t t1 order by (t1.b = 1 and exists (select 1 from t t2 where t1.b = t2.b)) limit 1; +id estRows task access object operator info +Projection 1.00 root planner__core__casetest__integration.t.a +└─Projection 1.00 root planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.b, Column#11 + └─TopN 1.00 root Column#13, offset:0, count:1 + └─Projection 10000.00 root planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.b, Column#11, and(eq(planner__core__casetest__integration.t.b, 1), Column#11)->Column#13 + └─HashJoin 10000.00 root left outer semi join, equal:[eq(planner__core__casetest__integration.t.b, planner__core__casetest__integration.t.b)] + ├─TableReader(Build) 10000.00 root data:TableFullScan + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +desc format = 'brief' select * from (select b+b as x from t) t1, t t2 where t1.x=t2.b order by t1.x limit 1; +id estRows task access object operator info +TopN 1.00 root Column#4, offset:0, count:1 +└─Projection 10000.00 root Column#4, planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.b + └─HashJoin 10000.00 root inner join, equal:[eq(planner__core__casetest__integration.t.b, Column#4)] + ├─Projection(Build) 8000.00 root plus(planner__core__casetest__integration.t.b, planner__core__casetest__integration.t.b)->Column#4 + │ └─TableReader 8000.00 root data:Selection + │ └─Selection 8000.00 cop[tikv] not(isnull(plus(planner__core__casetest__integration.t.b, planner__core__casetest__integration.t.b))) + │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.b)) + └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +drop table if exists t; +create table t(a int, b int); +insert into t values(1, 1), (2, 1), (3, 2), (4, 2), (5, 2); +explain select approx_percentile(a, 50) from t; +id estRows task access object operator info +HashAgg_5 1.00 root funcs:approx_percentile(planner__core__casetest__integration.t.a, 50)->Column#4 +└─TableReader_11 10000.00 root data:TableFullScan_10 + └─TableFullScan_10 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select approx_percentile(a, 50) from t; +approx_percentile(a, 50) +3 +explain select approx_percentile(a, 10) from t; +id estRows task access object operator info +HashAgg_5 1.00 root funcs:approx_percentile(planner__core__casetest__integration.t.a, 10)->Column#4 +└─TableReader_11 10000.00 root data:TableFullScan_10 + └─TableFullScan_10 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select approx_percentile(a, 10) from t; +approx_percentile(a, 10) +1 +explain select approx_percentile(a, 10+70) from t; +id estRows task access object operator info +HashAgg_5 1.00 root funcs:approx_percentile(planner__core__casetest__integration.t.a, 80)->Column#4 +└─TableReader_11 10000.00 root data:TableFullScan_10 + └─TableFullScan_10 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select approx_percentile(a, 10+70) from t; +approx_percentile(a, 10+70) +4 +explain select approx_percentile(a, 10*10) from t; +id estRows task access object operator info +HashAgg_5 1.00 root funcs:approx_percentile(planner__core__casetest__integration.t.a, 100)->Column#4 +└─TableReader_11 10000.00 root data:TableFullScan_10 + └─TableFullScan_10 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select approx_percentile(a, 10*10) from t; +approx_percentile(a, 10*10) +5 +explain select approx_percentile(a, 50) from t group by b order by b; +id estRows task access object operator info +Projection_6 8000.00 root Column#4->Column#5 +└─Sort_7 8000.00 root planner__core__casetest__integration.t.b + └─HashAgg_9 8000.00 root group by:planner__core__casetest__integration.t.b, funcs:approx_percentile(planner__core__casetest__integration.t.a, 50)->Column#4, funcs:firstrow(planner__core__casetest__integration.t.b)->planner__core__casetest__integration.t.b + └─TableReader_13 10000.00 root data:TableFullScan_12 + └─TableFullScan_12 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select approx_percentile(a, 50) from t group by b order by b; +approx_percentile(a, 50) +1 +4 +drop table if exists t; +create table t(a int); +insert into t values(1),(1),(2); +explain format = 'brief' select /*+ stream_agg() */ count(*) c from t group by a order by c limit 1; +id estRows task access object operator info +TopN 1.00 root Column#3, offset:0, count:1 +└─StreamAgg 8000.00 root group by:planner__core__casetest__integration.t.a, funcs:count(1)->Column#3 + └─Sort 10000.00 root planner__core__casetest__integration.t.a + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select /*+ stream_agg() */ count(*) c from t group by a order by c limit 1; +c +1 +explain format = 'brief' select /*+ stream_agg() */ count(*) c from t group by a order by c; +id estRows task access object operator info +Sort 8000.00 root Column#3 +└─StreamAgg 8000.00 root group by:planner__core__casetest__integration.t.a, funcs:count(1)->Column#3 + └─Sort 10000.00 root planner__core__casetest__integration.t.a + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select /*+ stream_agg() */ count(*) c from t group by a order by c; +c +1 +2 +explain format = 'brief' select /*+ stream_agg() */ count(*) c from t group by a order by a limit 1; +id estRows task access object operator info +Projection 1.00 root Column#3->Column#4 +└─Limit 1.00 root offset:0, count:1 + └─StreamAgg 1.00 root group by:planner__core__casetest__integration.t.a, funcs:count(1)->Column#3, funcs:firstrow(planner__core__casetest__integration.t.a)->planner__core__casetest__integration.t.a + └─Sort 1.25 root planner__core__casetest__integration.t.a + └─TableReader 1.25 root data:TableFullScan + └─TableFullScan 1.25 cop[tikv] table:t keep order:false, stats:pseudo +select /*+ stream_agg() */ count(*) c from t group by a order by a limit 1; +c +2 +explain format = 'brief' select /*+ stream_agg() */ count(*) c from t group by a order by a; +id estRows task access object operator info +Projection 8000.00 root Column#3->Column#4 +└─StreamAgg 8000.00 root group by:planner__core__casetest__integration.t.a, funcs:count(1)->Column#3, funcs:firstrow(planner__core__casetest__integration.t.a)->planner__core__casetest__integration.t.a + └─Sort 10000.00 root planner__core__casetest__integration.t.a + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select /*+ stream_agg() */ count(*) c from t group by a order by a; +c +2 +1 +drop table if exists t; +drop table if exists s; +create table t(a int, b int); +create table s(a int, b int, index(a)); +insert into t values(1,1),(1,2),(2,2); +insert into s values(1,1),(2,2),(2,1); +explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.b; +id estRows task access object operator info +IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t.a, inner key:planner__core__casetest__integration.s.a, equal cond:eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.a), eq(planner__core__casetest__integration.t.b, planner__core__casetest__integration.s.b) +├─TableReader(Build) 9980.01 root data:Selection +│ └─Selection 9980.01 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)), not(isnull(planner__core__casetest__integration.t.b)) +│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +└─IndexLookUp(Probe) 12475.01 root + ├─Selection(Build) 12487.50 cop[tikv] not(isnull(planner__core__casetest__integration.s.a)) + │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(planner__core__casetest__integration.s.a, planner__core__casetest__integration.t.a)], keep order:false, stats:pseudo + └─Selection(Probe) 12475.01 cop[tikv] not(isnull(planner__core__casetest__integration.s.b)) + └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo +explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.a; +id estRows task access object operator info +IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t.a, inner key:planner__core__casetest__integration.s.a, equal cond:eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.a), eq(planner__core__casetest__integration.t.b, planner__core__casetest__integration.s.a) +├─TableReader(Build) 9980.01 root data:Selection +│ └─Selection 9980.01 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)), not(isnull(planner__core__casetest__integration.t.b)) +│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +└─IndexLookUp(Probe) 12475.01 root + ├─Selection(Build) 12475.01 cop[tikv] not(isnull(planner__core__casetest__integration.s.a)) + │ └─IndexRangeScan 12487.50 cop[tikv] table:s, index:a(a) range: decided by [eq(planner__core__casetest__integration.s.a, planner__core__casetest__integration.t.a)], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:s keep order:false, stats:pseudo +explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.a = s.b; +id estRows task access object operator info +Projection 12475.01 root planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.b, planner__core__casetest__integration.s.a, planner__core__casetest__integration.s.b +└─IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t.a, inner key:planner__core__casetest__integration.s.a, equal cond:eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.a), eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.b) + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)) + │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo + └─IndexLookUp(Probe) 12475.01 root + ├─Selection(Build) 12487.50 cop[tikv] not(isnull(planner__core__casetest__integration.s.a)) + │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(planner__core__casetest__integration.s.a, planner__core__casetest__integration.t.a)], keep order:false, stats:pseudo + └─Selection(Probe) 12475.01 cop[tikv] not(isnull(planner__core__casetest__integration.s.b)) + └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo +explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.b; +id estRows task access object operator info +IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t.a, inner key:planner__core__casetest__integration.s.a, equal cond:eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.a), eq(planner__core__casetest__integration.t.b, planner__core__casetest__integration.s.b) +├─TableReader(Build) 9980.01 root data:Selection +│ └─Selection 9980.01 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)), not(isnull(planner__core__casetest__integration.t.b)) +│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +└─IndexLookUp(Probe) 12475.01 root + ├─Selection(Build) 12487.50 cop[tikv] not(isnull(planner__core__casetest__integration.s.a)) + │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(planner__core__casetest__integration.s.a, planner__core__casetest__integration.t.a)], keep order:false, stats:pseudo + └─Selection(Probe) 12475.01 cop[tikv] not(isnull(planner__core__casetest__integration.s.b)) + └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo +explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.a; +id estRows task access object operator info +IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t.a, inner key:planner__core__casetest__integration.s.a, equal cond:eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.a), eq(planner__core__casetest__integration.t.b, planner__core__casetest__integration.s.a) +├─TableReader(Build) 9980.01 root data:Selection +│ └─Selection 9980.01 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)), not(isnull(planner__core__casetest__integration.t.b)) +│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +└─IndexLookUp(Probe) 12475.01 root + ├─Selection(Build) 12475.01 cop[tikv] not(isnull(planner__core__casetest__integration.s.a)) + │ └─IndexRangeScan 12487.50 cop[tikv] table:s, index:a(a) range: decided by [eq(planner__core__casetest__integration.s.a, planner__core__casetest__integration.t.a)], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:s keep order:false, stats:pseudo +explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.a = s.b; +id estRows task access object operator info +Projection 12475.01 root planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.b, planner__core__casetest__integration.s.a, planner__core__casetest__integration.s.b +└─IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t.a, inner key:planner__core__casetest__integration.s.a, equal cond:eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.a), eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.b) + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)) + │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo + └─IndexLookUp(Probe) 12475.01 root + ├─Selection(Build) 12487.50 cop[tikv] not(isnull(planner__core__casetest__integration.s.a)) + │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(planner__core__casetest__integration.s.a, planner__core__casetest__integration.t.a)], keep order:false, stats:pseudo + └─Selection(Probe) 12475.01 cop[tikv] not(isnull(planner__core__casetest__integration.s.b)) + └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo +drop table if exists t; +create table t(a int, b int); +insert into t values(1, 2), (3, 4); +explain format = 'brief' select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t; +id estRows task access object operator info +HashJoin 10000.00 root CARTESIAN left outer semi join, other cond:eq(2, planner__core__casetest__integration.t.b) +├─TableReader(Build) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +└─Projection(Probe) 10000.00 root 1->Column#22 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t; +(2) in (select b from t) +1 +1 +drop table if exists t1; +create table t1 (c1 int primary key, c2 int, c3 int, index c2 (c2)); +select count(1) from (select count(1) from (select * from t1 where c3 = 100) k) k2; +count(1) +1 +set tidb_cost_model_version=2; +drop table if exists t1,t2,t3; +create table t1 (pk char(32) primary key nonclustered, col1 char(32), col2 varchar(40), col3 char(32), key (col1), key (col3), key (col2,col3), key (col1,col3)); +create table t2 (pk char(32) primary key nonclustered, col1 varchar(100)); +create table t3 (pk char(32) primary key nonclustered, keycol varchar(100), pad1 tinyint(1) default null, pad2 varchar(40), key (keycol,pad1,pad2)); +explain format = 'brief' SELECT t1.pk FROM t1 INNER JOIN t2 ON t1.col1 = t2.pk INNER JOIN t3 ON t1.col3 = t3.pk WHERE t2.col1 IN ('a' , 'b') AND t3.keycol = 'c' AND t1.col2 = 'a' AND t1.col1 != 'abcdef' AND t1.col1 != 'aaaaaa'; +id estRows task access object operator info +IndexHashJoin 13.81 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t1.col1, inner key:planner__core__casetest__integration.t2.pk, equal cond:eq(planner__core__casetest__integration.t1.col1, planner__core__casetest__integration.t2.pk) +├─IndexHashJoin(Build) 12.50 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t3.pk, inner key:planner__core__casetest__integration.t1.col3, equal cond:eq(planner__core__casetest__integration.t3.pk, planner__core__casetest__integration.t1.col3) +│ ├─IndexLookUp(Build) 10.00 root +│ │ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t3, index:keycol(keycol, pad1, pad2) range:["c","c"], keep order:false, stats:pseudo +│ │ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t3 keep order:false, stats:pseudo +│ └─IndexLookUp(Probe) 12.50 root +│ ├─Selection(Build) 18.10 cop[tikv] not(isnull(planner__core__casetest__integration.t1.col3)) +│ │ └─IndexRangeScan 18.12 cop[tikv] table:t1, index:col2(col2, col3) range: decided by [eq(planner__core__casetest__integration.t1.col3, planner__core__casetest__integration.t3.pk) eq(planner__core__casetest__integration.t1.col2, a)], keep order:false, stats:pseudo +│ └─Selection(Probe) 12.50 cop[tikv] ne(planner__core__casetest__integration.t1.col1, "aaaaaa"), ne(planner__core__casetest__integration.t1.col1, "abcdef"), not(isnull(planner__core__casetest__integration.t1.col1)) +│ └─TableRowIDScan 18.10 cop[tikv] table:t1 keep order:false, stats:pseudo +└─IndexLookUp(Probe) 12.50 root + ├─Selection(Build) 12.50 cop[tikv] ne(planner__core__casetest__integration.t2.pk, "aaaaaa"), ne(planner__core__casetest__integration.t2.pk, "abcdef") + │ └─IndexRangeScan 12.50 cop[tikv] table:t2, index:PRIMARY(pk) range: decided by [eq(planner__core__casetest__integration.t2.pk, planner__core__casetest__integration.t1.col1)], keep order:false, stats:pseudo + └─Selection(Probe) 12.50 cop[tikv] in(planner__core__casetest__integration.t2.col1, "a", "b") + └─TableRowIDScan 12.50 cop[tikv] table:t2 keep order:false, stats:pseudo +explain format = 'brief' SELECT t1.pk FROM t1 LEFT JOIN t2 ON t1.col1 = t2.pk LEFT JOIN t3 ON t1.col3 = t3.pk WHERE t2.col1 IN ('a' , 'b') AND t3.keycol = 'c' AND t1.col2 = 'a' AND t1.col1 != 'abcdef' AND t1.col1 != 'aaaaaa'; +id estRows task access object operator info +IndexHashJoin 13.81 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t1.col1, inner key:planner__core__casetest__integration.t2.pk, equal cond:eq(planner__core__casetest__integration.t1.col1, planner__core__casetest__integration.t2.pk) +├─IndexHashJoin(Build) 12.50 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t3.pk, inner key:planner__core__casetest__integration.t1.col3, equal cond:eq(planner__core__casetest__integration.t3.pk, planner__core__casetest__integration.t1.col3) +│ ├─IndexLookUp(Build) 10.00 root +│ │ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t3, index:keycol(keycol, pad1, pad2) range:["c","c"], keep order:false, stats:pseudo +│ │ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t3 keep order:false, stats:pseudo +│ └─IndexLookUp(Probe) 12.50 root +│ ├─Selection(Build) 18.10 cop[tikv] not(isnull(planner__core__casetest__integration.t1.col3)) +│ │ └─IndexRangeScan 18.12 cop[tikv] table:t1, index:col2(col2, col3) range: decided by [eq(planner__core__casetest__integration.t1.col3, planner__core__casetest__integration.t3.pk) eq(planner__core__casetest__integration.t1.col2, a)], keep order:false, stats:pseudo +│ └─Selection(Probe) 12.50 cop[tikv] ne(planner__core__casetest__integration.t1.col1, "aaaaaa"), ne(planner__core__casetest__integration.t1.col1, "abcdef"), not(isnull(planner__core__casetest__integration.t1.col1)) +│ └─TableRowIDScan 18.10 cop[tikv] table:t1 keep order:false, stats:pseudo +└─IndexLookUp(Probe) 12.50 root + ├─Selection(Build) 12.50 cop[tikv] ne(planner__core__casetest__integration.t2.pk, "aaaaaa"), ne(planner__core__casetest__integration.t2.pk, "abcdef") + │ └─IndexRangeScan 12.50 cop[tikv] table:t2, index:PRIMARY(pk) range: decided by [eq(planner__core__casetest__integration.t2.pk, planner__core__casetest__integration.t1.col1)], keep order:false, stats:pseudo + └─Selection(Probe) 12.50 cop[tikv] in(planner__core__casetest__integration.t2.col1, "a", "b") + └─TableRowIDScan 12.50 cop[tikv] table:t2 keep order:false, stats:pseudo +drop table if exists t1, t2; +create table t1(a int, b int, c int, d int, index idx_a_b_c(a, b, c)); +create table t2(a int, b int, c int, d int, index idx_a_b_c_d(a, b, c, d)); +explain format = 'brief' select a, b, c from t1 where a > 3 and b = 4 order by a, c; +id estRows task access object operator info +IndexReader 3.33 root index:Selection +└─Selection 3.33 cop[tikv] eq(planner__core__casetest__integration.t1.b, 4) + └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx_a_b_c(a, b, c) range:(3,+inf], keep order:true, stats:pseudo +explain format = 'brief' select * from t2 where a = 1 and c = 2 order by b, d; +id estRows task access object operator info +IndexReader 0.01 root index:Selection +└─Selection 0.01 cop[tikv] eq(planner__core__casetest__integration.t2.c, 2) + └─IndexRangeScan 10.00 cop[tikv] table:t2, index:idx_a_b_c_d(a, b, c, d) range:[1,1], keep order:true, stats:pseudo +explain format = 'brief' select a, b, c from t1 where (a = 1 and b = 1 and c = 1) or (a = 1 and b = 1 and c = 2) order by c; +id estRows task access object operator info +IndexReader 0.03 root index:IndexRangeScan +└─IndexRangeScan 0.03 cop[tikv] table:t1, index:idx_a_b_c(a, b, c) range:[1 1 1,1 1 2], keep order:true, stats:pseudo +explain format = 'brief' select a, b, c from t1 where (a = 1 and b = 1 and c < 3) or (a = 1 and b = 1 and c > 6) order by c; +id estRows task access object operator info +IndexReader 0.67 root index:IndexRangeScan +└─IndexRangeScan 0.67 cop[tikv] table:t1, index:idx_a_b_c(a, b, c) range:[1 1 -inf,1 1 3), (1 1 6,1 1 +inf], keep order:true, stats:pseudo +explain format = 'brief' select * from t2 where ((a = 1 and b = 1 and d < 3) or (a = 1 and b = 1 and d > 6)) and c = 3 order by d; +id estRows task access object operator info +IndexReader 0.00 root index:Selection +└─Selection 0.00 cop[tikv] eq(planner__core__casetest__integration.t2.c, 3), or(and(eq(planner__core__casetest__integration.t2.a, 1), and(eq(planner__core__casetest__integration.t2.b, 1), lt(planner__core__casetest__integration.t2.d, 3))), and(eq(planner__core__casetest__integration.t2.a, 1), and(eq(planner__core__casetest__integration.t2.b, 1), gt(planner__core__casetest__integration.t2.d, 6)))) + └─IndexRangeScan 10.00 cop[tikv] table:t2, index:idx_a_b_c_d(a, b, c, d) range:[1,1], keep order:true, stats:pseudo +drop table if exists t; +create table t(a int not null, b int not null); +explain format = 'brief' select * from t where exists (select 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a); +id estRows task access object operator info +HashJoin 8000.00 root semi join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +├─HashJoin(Build) 12500.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +│ ├─TableReader(Build) 10000.00 root data:TableFullScan +│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +│ └─TableReader(Probe) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where exists (select 1 from t t1 join t t2 on t1.a = t2.a and t1.a = t.a); +id estRows task access object operator info +HashJoin 8000.00 root semi join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +├─HashJoin(Build) 12500.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +│ ├─TableReader(Build) 10000.00 root data:TableFullScan +│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +│ └─TableReader(Probe) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a); +id estRows task access object operator info +HashJoin 10000.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +├─HashAgg(Build) 8000.00 root group by:planner__core__casetest__integration.t.a, funcs:firstrow(planner__core__casetest__integration.t.a)->planner__core__casetest__integration.t.a +│ └─HashJoin 12500.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +│ ├─TableReader(Build) 10000.00 root data:TableFullScan +│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +│ └─TableReader(Probe) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 on t1.a = t2.a and t1.a = t.a); +id estRows task access object operator info +HashJoin 10000.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +├─HashAgg(Build) 8000.00 root group by:planner__core__casetest__integration.t.a, funcs:firstrow(planner__core__casetest__integration.t.a)->planner__core__casetest__integration.t.a +│ └─HashJoin 12500.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +│ ├─TableReader(Build) 10000.00 root data:TableFullScan +│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +│ └─TableReader(Probe) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select /*+ hash_join_build(t) */ * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a); +id estRows task access object operator info +HashJoin 10000.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +├─HashAgg(Build) 8000.00 root group by:planner__core__casetest__integration.t.a, funcs:firstrow(planner__core__casetest__integration.t.a)->planner__core__casetest__integration.t.a +│ └─HashJoin 12500.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +│ ├─TableReader(Build) 10000.00 root data:TableFullScan +│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +│ └─TableReader(Probe) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select /*+ hash_join_probe(t) */ * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a); +id estRows task access object operator info +HashJoin 10000.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +├─HashAgg(Build) 8000.00 root group by:planner__core__casetest__integration.t.a, funcs:firstrow(planner__core__casetest__integration.t.a)->planner__core__casetest__integration.t.a +│ └─HashJoin 12500.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +│ ├─TableReader(Build) 10000.00 root data:TableFullScan +│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +│ └─TableReader(Probe) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +drop table if exists test; +create table test(id int, value int); +drop table if exists t; +create table t(c int); +insert t values(10), (8), (7), (9), (11); +explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id limit 1); +id estRows task access object operator info +HashAgg 1.00 root funcs:count(1)->Column#7 +└─HashJoin 7992.00 root semi join, equal:[eq(planner__core__casetest__integration.test.id, planner__core__casetest__integration.test.id)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.test.id)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.test.id)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id); +id estRows task access object operator info +HashAgg 1.00 root funcs:count(1)->Column#7 +└─HashJoin 7992.00 root semi join, equal:[eq(planner__core__casetest__integration.test.id, planner__core__casetest__integration.test.id)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.test.id)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.test.id)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id limit 1,2); +id estRows task access object operator info +HashAgg 1.00 root funcs:count(1)->Column#7 +└─Apply 10000.00 root CARTESIAN semi join + ├─TableReader(Build) 10000.00 root data:TableFullScan + │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo + └─Limit(Probe) 20000.00 root offset:1, count:2 + └─TableReader 30000.00 root data:Limit + └─Limit 30000.00 cop[tikv] offset:0, count:3 + └─Selection 30000.00 cop[tikv] eq(planner__core__casetest__integration.test.id, planner__core__casetest__integration.test.id) + └─TableFullScan 30000000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +explain format = 'brief' select * from t where 9 in (select c from t s where s.c < t.c limit 3); +id estRows task access object operator info +Apply 10000.00 root CARTESIAN semi join +├─TableReader(Build) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +└─Selection(Probe) 24000.00 root eq(9, planner__core__casetest__integration.t.c) + └─Limit 30000.00 root offset:0, count:3 + └─TableReader 30000.00 root data:Limit + └─Limit 30000.00 cop[tikv] offset:0, count:3 + └─Selection 30000.00 cop[tikv] lt(planner__core__casetest__integration.t.c, planner__core__casetest__integration.t.c) + └─TableFullScan 37500.00 cop[tikv] table:s keep order:false, stats:pseudo +drop table if exists t0; +create table t0 (a int, b int, index(a, b)); +insert into t0 values (1, 1); +insert into t0 values (2, 2); +insert into t0 values (2, 2); +insert into t0 values (2, 2); +insert into t0 values (2, 2); +insert into t0 values (2, 2); +insert into t0 values (3, 3); +drop table if exists t1; +create table t1 (a int, b int, c int, index(a, b, c)); +drop table if exists t2; +create table t2 (a float, b float, index(a, b)); +drop table if exists t3; +create table t3 (a char(10), b char(10), c char(10), index(a, b, c)); +explain format = 'brief' select * from t0 where a > 1 and a < 3 order by b limit 2; +id estRows task access object operator info +Limit 2.00 root offset:0, count:2 +└─IndexReader 2.00 root index:Limit + └─Limit 2.00 cop[tikv] offset:0, count:2 + └─IndexRangeScan 2.50 cop[tikv] table:t0, index:a(a, b) range:[2,2], keep order:true, stats:pseudo +explain format = 'brief' select * from t1 where a >= 2 and a <= 2 and b = 2 and c > 2; +id estRows task access object operator info +IndexReader 0.33 root index:IndexRangeScan +└─IndexRangeScan 0.33 cop[tikv] table:t1, index:a(a, b, c) range:(2 2 2,2 2 +inf], keep order:false, stats:pseudo +explain format = 'brief' select * from t2 where a >= 2.5 and a <= 2.5 order by b limit 2; +id estRows task access object operator info +Limit 2.00 root offset:0, count:2 +└─IndexReader 2.00 root index:Limit + └─Limit 2.00 cop[tikv] offset:0, count:2 + └─IndexRangeScan 2.00 cop[tikv] table:t2, index:a(a, b) range:[2.5,2.5], keep order:true, stats:pseudo +explain format = 'brief' select * from t3 where a >= 'a' and a <= 'a' and b = 'b' and c > 'c'; +id estRows task access object operator info +IndexReader 0.33 root index:IndexRangeScan +└─IndexRangeScan 0.33 cop[tikv] table:t3, index:a(a, b, c) range:("a" "b" "c","a" "b" +inf], keep order:false, stats:pseudo +drop table if exists t1; +CREATE TABLE t1 ( +key1 int(11) NOT NULL, +key2 int(11) NOT NULL, +key3 int(11) NOT NULL, +key4 int(11) NOT NULL, +key5 int(11) DEFAULT NULL, +key6 int(11) DEFAULT NULL, +key7 int(11) NOT NULL, +key8 int(11) NOT NULL, +KEY i1 (key1), +KEY i2 (key2), +KEY i3 (key3), +KEY i4 (key4), +KEY i5 (key5), +KEY i6 (key6) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +explain format = 'brief' SELECT /*+ use_index_merge(t1)*/ COUNT(*) FROM t1 WHERE (key4=42 AND key6 IS NOT NULL) OR (key1=4 AND key3=6); +id estRows task access object operator info +StreamAgg 1.00 root funcs:count(1)->Column#10 +└─IndexMerge 0.02 root type: union + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:i4(key4) range:[42,42], keep order:false, stats:pseudo + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:i1(key1) range:[4,4], keep order:false, stats:pseudo + └─Selection(Probe) 0.02 cop[tikv] or(and(eq(planner__core__casetest__integration.t1.key4, 42), not(isnull(planner__core__casetest__integration.t1.key6))), and(eq(planner__core__casetest__integration.t1.key1, 4), eq(planner__core__casetest__integration.t1.key3, 6))) + └─TableRowIDScan 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo +drop table if exists tpk; +create table tuk (a int, b int, c int, unique key (a, b, c)); +create table tik (a int, b int, c int, key (a, b, c)); +insert into tuk values (NULL, NULL, NULL); +insert into tik values (NULL, NULL, NULL); +insert into tuk values (NULL, NULL, NULL); +insert into tik values (NULL, NULL, NULL); +insert into tuk values (NULL, NULL, 1); +insert into tik values (NULL, NULL, 1); +insert into tuk values (NULL, NULL, 1); +insert into tik values (NULL, NULL, 1); +insert into tuk values (NULL, 1, NULL); +insert into tik values (NULL, 1, NULL); +insert into tuk values (NULL, 1, NULL); +insert into tik values (NULL, 1, NULL); +insert into tuk values (NULL, 1, 1); +insert into tik values (NULL, 1, 1); +insert into tuk values (NULL, 1, 1); +insert into tik values (NULL, 1, 1); +insert into tuk values (1, NULL, NULL); +insert into tik values (1, NULL, NULL); +insert into tuk values (1, NULL, NULL); +insert into tik values (1, NULL, NULL); +insert into tuk values (1, NULL, 1); +insert into tik values (1, NULL, 1); +insert into tuk values (1, NULL, 1); +insert into tik values (1, NULL, 1); +insert into tuk values (1, 1, NULL); +insert into tik values (1, 1, NULL); +insert into tuk values (1, 1, NULL); +insert into tik values (1, 1, NULL); +insert into tuk values (1, 1, 1); +insert into tik values (1, 1, 1); +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b=1; +id estRows task access object operator info +IndexReader_6 0.10 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo +select * from tuk where a<=>null and b=1; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b=1; +id estRows task access object operator info +IndexReader_7 0.01 root index:Selection_6 +└─Selection_6 0.01 cop[tikv] eq(planner__core__casetest__integration.tuk.b, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tuk where a<=>null and b=1; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b=1; +id estRows task access object operator info +IndexReader_6 0.10 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo +select * from tik where a<=>null and b=1; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b=1; +id estRows task access object operator info +IndexReader_7 0.01 root index:Selection_6 +└─Selection_6 0.01 cop[tikv] eq(planner__core__casetest__integration.tik.b, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tik where a<=>null and b=1; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b>0 and b<2; +id estRows task access object operator info +IndexReader_6 0.10 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo +select * from tuk where a<=>null and b>0 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b>0 and b<2; +id estRows task access object operator info +IndexReader_7 0.25 root index:Selection_6 +└─Selection_6 0.25 cop[tikv] eq(planner__core__casetest__integration.tuk.b, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tuk where a<=>null and b>0 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b>0 and b<2; +id estRows task access object operator info +IndexReader_6 0.10 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo +select * from tik where a<=>null and b>0 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b>0 and b<2; +id estRows task access object operator info +IndexReader_7 0.25 root index:Selection_6 +└─Selection_6 0.25 cop[tikv] eq(planner__core__casetest__integration.tik.b, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tik where a<=>null and b>0 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b>=1 and b<2; +id estRows task access object operator info +IndexReader_6 0.10 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo +select * from tuk where a<=>null and b>=1 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b>=1 and b<2; +id estRows task access object operator info +IndexReader_7 0.25 root index:Selection_6 +└─Selection_6 0.25 cop[tikv] eq(planner__core__casetest__integration.tuk.b, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tuk where a<=>null and b>=1 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b>=1 and b<2; +id estRows task access object operator info +IndexReader_6 0.10 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo +select * from tik where a<=>null and b>=1 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b>=1 and b<2; +id estRows task access object operator info +IndexReader_7 0.25 root index:Selection_6 +└─Selection_6 0.25 cop[tikv] eq(planner__core__casetest__integration.tik.b, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tik where a<=>null and b>=1 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b=1 and c=1; +id estRows task access object operator info +IndexReader_6 1.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1 1,NULL 1 1], keep order:false, stats:pseudo +select * from tuk where a<=>null and b=1 and c=1; +a b c +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b=1 and c=1; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] eq(planner__core__casetest__integration.tuk.b, 1), eq(planner__core__casetest__integration.tuk.c, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tuk where a<=>null and b=1 and c=1; +a b c +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b=1 and c=1; +id estRows task access object operator info +IndexReader_6 0.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1 1,NULL 1 1], keep order:false, stats:pseudo +select * from tik where a<=>null and b=1 and c=1; +a b c +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b=1 and c=1; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] eq(planner__core__casetest__integration.tik.b, 1), eq(planner__core__casetest__integration.tik.c, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tik where a<=>null and b=1 and c=1; +a b c +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a=1 and b<=>null and c=1; +id estRows task access object operator info +IndexReader_6 1.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[1 NULL 1,1 NULL 1], keep order:false, stats:pseudo +select * from tuk where a=1 and b<=>null and c=1; +a b c +1 NULL 1 +1 NULL 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a=1 and b<=>null and c=1; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] eq(planner__core__casetest__integration.tuk.c, 1) + └─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[1 NULL,1 NULL], keep order:false, stats:pseudo +select * from tuk where a=1 and b<=>null and c=1; +a b c +1 NULL 1 +1 NULL 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a=1 and b<=>null and c=1; +id estRows task access object operator info +IndexReader_6 0.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[1 NULL 1,1 NULL 1], keep order:false, stats:pseudo +select * from tik where a=1 and b<=>null and c=1; +a b c +1 NULL 1 +1 NULL 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a=1 and b<=>null and c=1; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] eq(planner__core__casetest__integration.tik.c, 1) + └─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[1 NULL,1 NULL], keep order:false, stats:pseudo +select * from tik where a=1 and b<=>null and c=1; +a b c +1 NULL 1 +1 NULL 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b<=>null and c=1; +id estRows task access object operator info +IndexReader_6 1.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL NULL 1,NULL NULL 1], keep order:false, stats:pseudo +select * from tuk where a<=>null and b<=>null and c=1; +a b c +NULL NULL 1 +NULL NULL 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b<=>null and c=1; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] eq(planner__core__casetest__integration.tuk.c, 1), nulleq(planner__core__casetest__integration.tuk.b, NULL) + └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tuk where a<=>null and b<=>null and c=1; +a b c +NULL NULL 1 +NULL NULL 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b<=>null and c=1; +id estRows task access object operator info +IndexReader_6 0.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL NULL 1,NULL NULL 1], keep order:false, stats:pseudo +select * from tik where a<=>null and b<=>null and c=1; +a b c +NULL NULL 1 +NULL NULL 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b<=>null and c=1; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] eq(planner__core__casetest__integration.tik.c, 1), nulleq(planner__core__casetest__integration.tik.b, NULL) + └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tik where a<=>null and b<=>null and c=1; +a b c +NULL NULL 1 +NULL NULL 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b<=>null and c<=>null; +id estRows task access object operator info +IndexReader_6 1.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL NULL NULL,NULL NULL NULL], keep order:false, stats:pseudo +select * from tuk where a<=>null and b<=>null and c<=>null; +a b c +NULL NULL NULL +NULL NULL NULL +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b<=>null and c<=>null; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] nulleq(planner__core__casetest__integration.tuk.b, NULL), nulleq(planner__core__casetest__integration.tuk.c, NULL) + └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tuk where a<=>null and b<=>null and c<=>null; +a b c +NULL NULL NULL +NULL NULL NULL +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b<=>null and c<=>null; +id estRows task access object operator info +IndexReader_6 0.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL NULL NULL,NULL NULL NULL], keep order:false, stats:pseudo +select * from tik where a<=>null and b<=>null and c<=>null; +a b c +NULL NULL NULL +NULL NULL NULL +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b<=>null and c<=>null; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] nulleq(planner__core__casetest__integration.tik.b, NULL), nulleq(planner__core__casetest__integration.tik.c, NULL) + └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tik where a<=>null and b<=>null and c<=>null; +a b c +NULL NULL NULL +NULL NULL NULL +set @@session.tidb_regard_null_as_point=default; +drop table if exists t1; +create table t1(c1 varchar(100), c2 varchar(100), key(c1), key(c2), c3 varchar(100)); +insert into t1 values('ab', '10', '10'); +drop table if exists tt1; +create table tt1(c1 varchar(100), c2 varchar(100), c3 varchar(100), c4 varchar(100), key idx_0(c1), key idx_1(c2, c3)); +insert into tt1 values('ab', '10', '10', '10'); +drop table if exists tt2; +create table tt2 (c1 int , pk int, primary key( pk ) , unique key( c1)); +insert into tt2 values(-3896405, -1), (-2, 1), (-1, -2); +drop table if exists tt3; +create table tt3(c1 int, c2 int, c3 int as (c1 + c2), key(c1), key(c2), key(c3)); +insert into tt3(c1, c2) values(1, 1); +select @@tidb_enable_index_merge; +@@tidb_enable_index_merge +1 +set tidb_enable_index_merge = on; +explain format=brief select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab'; +id estRows task access object operator info +Projection 15.99 root 1->Column#5 +└─Selection 15.99 root or(eq(planner__core__casetest__integration.t1.c1, "de"), and(eq(planner__core__casetest__integration.t1.c2, "10"), eq(from_base64(to_base64(planner__core__casetest__integration.t1.c1)), "ab"))) + └─IndexMerge 19.99 root type: union + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c1(c1) range:["de","de"], keep order:false, stats:pseudo + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c2(c2) range:["10","10"], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo +select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab'; +1 +1 +explain format=brief select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'ab' or c2 = '10' and char_length(left(c1, 10)) = 10; +id estRows task access object operator info +Projection 17.99 root 1->Column#5 +└─Selection 0.04 root or(eq(planner__core__casetest__integration.t1.c1, "ab"), and(eq(planner__core__casetest__integration.t1.c2, "10"), eq(char_length(left(planner__core__casetest__integration.t1.c1, 10)), 10))) + └─IndexMerge 19.99 root type: union + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c1(c1) range:["ab","ab"], keep order:false, stats:pseudo + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c2(c2) range:["10","10"], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo +select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'ab' or c2 = '10' and char_length(left(c1, 10)) = 10; +1 +1 +explain format=brief select /*+ use_index_merge(tt1) */ 1 from tt1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c3)) = '10'; +id estRows task access object operator info +Projection 15.99 root 1->Column#6 +└─Selection 15.99 root or(eq(planner__core__casetest__integration.tt1.c1, "de"), and(eq(planner__core__casetest__integration.tt1.c2, "10"), eq(from_base64(to_base64(planner__core__casetest__integration.tt1.c3)), "10"))) + └─IndexMerge 19.99 root type: union + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:tt1, index:idx_0(c1) range:["de","de"], keep order:false, stats:pseudo + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:tt1, index:idx_1(c2, c3) range:["10","10"], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 19.99 cop[tikv] table:tt1 keep order:false, stats:pseudo +select /*+ use_index_merge(tt1) */ 1 from tt1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c3)) = '10'; +1 +1 +explain format=brief select /*+ use_index_merge( tt2 ) */ 1 from tt2 where tt2.c1 in (-3896405) or tt2.pk in (1, 53330) and to_base64(left(pk, 5)); +id estRows task access object operator info +Projection 2.40 root 1->Column#3 +└─Selection 2.40 root or(eq(planner__core__casetest__integration.tt2.c1, -3896405), and(in(planner__core__casetest__integration.tt2.pk, 1, 53330), istrue_with_null(cast(to_base64(left(cast(planner__core__casetest__integration.tt2.pk, var_string(20)), 5)), double BINARY)))) + └─IndexMerge 3.00 root type: union + ├─IndexRangeScan(Build) 1.00 cop[tikv] table:tt2, index:c1(c1) range:[-3896405,-3896405], keep order:false, stats:pseudo + ├─TableRangeScan(Build) 2.00 cop[tikv] table:tt2 range:[1,1], [53330,53330], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 3.00 cop[tikv] table:tt2 keep order:false, stats:pseudo +select /*+ use_index_merge( tt2 ) */ 1 from tt2 where tt2.c1 in (-3896405) or tt2.pk in (1, 53330) and to_base64(left(pk, 5)); +1 +1 +explain format=brief select /*+ use_index_merge(tt3) */ 1 from tt3 where c1 < -10 or c2 < 10 and reverse(c3) = '2'; +id estRows task access object operator info +Projection 5098.44 root 1->Column#5 +└─Selection 2825.66 root or(lt(planner__core__casetest__integration.tt3.c1, -10), and(lt(planner__core__casetest__integration.tt3.c2, 10), eq(reverse(cast(planner__core__casetest__integration.tt3.c3, var_string(20))), "2"))) + └─IndexMerge 5542.21 root type: union + ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:tt3, index:c1(c1) range:[-inf,-10), keep order:false, stats:pseudo + ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:tt3, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 5542.21 cop[tikv] table:tt3 keep order:false, stats:pseudo +select /*+ use_index_merge(tt3) */ 1 from tt3 where c1 < -10 or c2 < 10 and reverse(c3) = '2'; +1 +1 +explain format=brief select 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab'; +id estRows task access object operator info +Projection 8000.00 root 1->Column#5 +└─Selection 8000.00 root or(eq(planner__core__casetest__integration.t1.c1, "de"), and(eq(planner__core__casetest__integration.t1.c2, "10"), eq(from_base64(to_base64(planner__core__casetest__integration.t1.c1)), "ab"))) + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +select 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab'; +1 +1 +set tidb_enable_index_merge = 1; +set tidb_enable_index_merge = default; +drop table if exists t1,t2; +create table t1(a int); +create table t2(a int, b int, c int, primary key(a,b) nonclustered); +explain format = 'brief' select (select c from t2 where t2.a = t1.a and t2.b = 1) from t1; +id estRows task access object operator info +HashJoin 10000.00 root left outer join, equal:[eq(planner__core__casetest__integration.t1.a, planner__core__casetest__integration.t2.a)] +├─TableReader(Build) 10.00 root data:Selection +│ └─Selection 10.00 cop[tikv] eq(planner__core__casetest__integration.t2.b, 1) +│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select (select c from t2 where t2.a = t1.a and (t2.b = 1 or t2.b = 2)) from t1; +id estRows task access object operator info +Projection 10000.00 root planner__core__casetest__integration.t2.c +└─Apply 10000.00 root CARTESIAN left outer join + ├─TableReader(Build) 10000.00 root data:TableFullScan + │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo + └─MaxOneRow(Probe) 10000.00 root + └─IndexLookUp 200.00 root + ├─Selection(Build) 200.00 cop[tikv] or(eq(planner__core__casetest__integration.t2.b, 1), eq(planner__core__casetest__integration.t2.b, 2)) + │ └─IndexRangeScan 100000.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(planner__core__casetest__integration.t2.a, planner__core__casetest__integration.t1.a)], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 200.00 cop[tikv] table:t2 keep order:false, stats:pseudo +set tidb_cost_model_version=2; +drop sequence if exists s1, s2; +create sequence s1; +create sequence s2; +explain format = 'brief' select 1 from s1; +id estRows task access object operator info +Projection 1.00 root 1->Column#1 +└─TableDual 1.00 root rows:1 +explain format = 'brief' select count(1) from s1; +id estRows task access object operator info +StreamAgg 1.00 root funcs:count(1)->Column#1 +└─TableDual 1.00 root rows:1 +explain format = 'brief' select count(*) from s1; +id estRows task access object operator info +StreamAgg 1.00 root funcs:count(1)->Column#1 +└─TableDual 1.00 root rows:1 +explain format = 'brief' select sum(1) from s1; +id estRows task access object operator info +StreamAgg 1.00 root funcs:sum(1)->Column#1 +└─TableDual 1.00 root rows:1 +explain format = 'brief' select count(1) as cnt from s1 union select count(1) as cnt from s2; +id estRows task access object operator info +HashAgg 2.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 2.00 root + ├─StreamAgg 1.00 root funcs:count(1)->Column#1 + │ └─TableDual 1.00 root rows:1 + └─StreamAgg 1.00 root funcs:count(1)->Column#2 + └─TableDual 1.00 root rows:1 +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int, b int, c int, d int, e int, f int, g int, primary key (a), unique key c_d_e (c, d, e), unique key f (f), unique key f_g (f, g), key g (g)); +set @@tidb_enable_chunk_rpc = on; +explain format = 'verbose' select * from t where a > 1 order by f; +id estRows estCost task access object operator info +Sort_5 3333.33 2146348.14 root planner__core__casetest__integration.t.f +└─TableReader_9 3333.33 160128.74 root data:TableRangeScan_8 + └─TableRangeScan_8 3333.33 923531.15 cop[tikv] table:t range:(1,+inf], keep order:false, stats:pseudo +Level Code Message +Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [{planner__core__casetest__integration.t.f asc}], TaskTp: rootTask} +Note 1105 [t] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask} +explain format = 'verbose' select * from t where f > 1; +id estRows estCost task access object operator info +TableReader_7 3333.33 316532.90 root data:Selection_6 +└─Selection_6 3333.33 3269593.45 cop[tikv] gt(planner__core__casetest__integration.t.f, 1) + └─TableFullScan_5 10000.00 2770593.45 cop[tikv] table:t keep order:false, stats:pseudo +Level Code Message +Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask} +explain format = 'verbose' select f from t where f > 1; +id estRows estCost task access object operator info +IndexReader_6 3333.33 50257.78 root index:IndexRangeScan_5 +└─IndexRangeScan_5 3333.33 542666.67 cop[tikv] table:t, index:f(f) range:(1,+inf], keep order:false, stats:pseudo +Level Code Message +Note 1105 [f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask} +explain format = 'verbose' select * from t where f > 3 and g = 5; +id estRows estCost task access object operator info +IndexLookUp_15 3.33 19551.99 root +├─IndexRangeScan_12(Build) 10.00 2035.00 cop[tikv] table:t, index:g(g) range:[5,5], keep order:false, stats:pseudo +└─Selection_14(Probe) 3.33 3269.59 cop[tikv] gt(planner__core__casetest__integration.t.f, 3) + └─TableRowIDScan_13 10.00 2770.59 cop[tikv] table:t keep order:false, stats:pseudo +Level Code Message +Note 1105 [t,f_g,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask} +explain format = 'verbose' select * from t where g = 5 order by f; +id estRows estCost task access object operator info +Sort_5 10.00 21321.97 root planner__core__casetest__integration.t.f +└─IndexLookUp_13 10.00 19545.34 root + ├─IndexRangeScan_11(Build) 10.00 2035.00 cop[tikv] table:t, index:g(g) range:[5,5], keep order:false, stats:pseudo + └─TableRowIDScan_12(Probe) 10.00 2770.59 cop[tikv] table:t keep order:false, stats:pseudo +Level Code Message +Note 1105 [t,f_g,g] remain after pruning paths for t given Prop{SortItems: [{planner__core__casetest__integration.t.f asc}], TaskTp: rootTask} +Note 1105 [t,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask} +explain format = 'verbose' select * from t where d = 3 order by c, e; +id estRows estCost task access object operator info +IndexLookUp_15 10.00 215519.24 root +├─Selection_14(Build) 10.00 2941000.00 cop[tikv] eq(planner__core__casetest__integration.t.d, 3) +│ └─IndexFullScan_12 10000.00 2442000.00 cop[tikv] table:t, index:c_d_e(c, d, e) keep order:true, stats:pseudo +└─TableRowIDScan_13(Probe) 10.00 2770.59 cop[tikv] table:t keep order:false, stats:pseudo +Level Code Message +Note 1105 [t,c_d_e] remain after pruning paths for t given Prop{SortItems: [{planner__core__casetest__integration.t.c asc} {planner__core__casetest__integration.t.e asc}], TaskTp: rootTask} +Note 1105 [t] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask} +set @@tidb_enable_chunk_rpc = default; +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int unsigned primary key, b int, c int, index idx_b(b)); +insert into t values (1,2,3), (4,5,6), (7,8,9), (10,11,12), (13,14,15); +analyze table t; +set @@tidb_enable_chunk_rpc = on; +set tidb_opt_prefer_range_scan = 0; +explain format = 'verbose' select * from t where b > 5; +id estRows estCost task access object operator info +TableReader_7 3.00 130.42 root data:Selection_6 +└─Selection_6 3.00 1386.04 cop[tikv] gt(planner__core__casetest__integration.t.b, 5) + └─TableFullScan_5 5.00 1136.54 cop[tikv] table:t keep order:false +explain format = 'verbose' select * from t where b = 6 order by a limit 1; +id estRows estCost task access object operator info +Limit_11 0.00 98.74 root offset:0, count:1 +└─TableReader_24 0.00 98.74 root data:Limit_23 + └─Limit_23 0.00 1386.04 cop[tikv] offset:0, count:1 + └─Selection_22 0.00 1386.04 cop[tikv] eq(planner__core__casetest__integration.t.b, 6) + └─TableFullScan_21 5.00 1136.54 cop[tikv] table:t keep order:true +explain format = 'verbose' select * from t where b = 6 limit 1; +id estRows estCost task access object operator info +Limit_8 0.00 98.74 root offset:0, count:1 +└─TableReader_13 0.00 98.74 root data:Limit_12 + └─Limit_12 0.00 1386.04 cop[tikv] offset:0, count:1 + └─Selection_11 0.00 1386.04 cop[tikv] eq(planner__core__casetest__integration.t.b, 6) + └─TableFullScan_10 5.00 1136.54 cop[tikv] table:t keep order:false +set tidb_opt_prefer_range_scan = 1; +explain format = 'verbose' select * from t where b > 5; +id estRows estCost task access object operator info +IndexLookUp_7 3.00 5856.46 root +├─IndexRangeScan_5(Build) 3.00 610.50 cop[tikv] table:t, index:idx_b(b) range:(5,+inf], keep order:false +└─TableRowIDScan_6(Probe) 3.00 681.92 cop[tikv] table:t keep order:false +Level Code Message +Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask} +explain format = 'verbose' select * from t where b = 6 order by a limit 1; +id estRows estCost task access object operator info +TopN_9 0.00 1956.63 root planner__core__casetest__integration.t.a, offset:0, count:1 +└─IndexLookUp_16 0.00 1951.83 root + ├─TopN_15(Build) 0.00 206.70 cop[tikv] planner__core__casetest__integration.t.a, offset:0, count:1 + │ └─IndexRangeScan_13 0.00 203.50 cop[tikv] table:t, index:idx_b(b) range:[6,6], keep order:false + └─TableRowIDScan_14(Probe) 0.00 186.61 cop[tikv] table:t keep order:false +Level Code Message +Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: copMultiReadTask} +explain format = 'verbose' select * from t where b = 6 limit 1; +id estRows estCost task access object operator info +IndexLookUp_13 0.00 1170.97 root limit embedded(offset:0, count:1) +├─Limit_12(Build) 0.00 203.50 cop[tikv] offset:0, count:1 +│ └─IndexRangeScan_10 0.00 203.50 cop[tikv] table:t, index:idx_b(b) range:[6,6], keep order:false +└─TableRowIDScan_11(Probe) 0.00 186.61 cop[tikv] table:t keep order:false +Level Code Message +Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: copMultiReadTask} +set @@tidb_enable_chunk_rpc = default; +set tidb_opt_prefer_range_scan = default; +drop table if exists t; +create table t(a int primary key, b int, c int, index idx_b(b)); +insert into t values (1,2,3), (4,5,6), (7,8,9), (10, 11, 12), (13,14,15), (16, 17, 18); +analyze table t; +explain format = 'brief' select * from t use index (idx_b) where b = 2 limit 1; +id estRows task access object operator info +IndexLookUp 1.00 root limit embedded(offset:0, count:1) +├─Limit(Build) 1.00 cop[tikv] offset:0, count:1 +│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:idx_b(b) range:[2,2], keep order:false +└─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false +drop table if exists t1; +create table t1(c1 int); +insert into t1 values(1), (2), (3), (4), (5), (6); +select floor(dt.rn/2) rownum, count(c1) from (select @rownum := @rownum + 1 rn, c1 from (select @rownum := -1) drn, t1) dt group by floor(dt.rn/2) order by rownum; +rownum count(c1) +0 2 +1 2 +2 2 +create table ta(a int, b int); +set sql_mode=''; +explain format = 'brief' select floor(dt.rn/2) rownum, count(c1) from (select @rownum := @rownum + 1 rn, c1 from (select @rownum := -1) drn, t1) dt group by floor(dt.rn/2) order by rownum; +id estRows task access object operator info +Sort 1.00 root Column#6 +└─Projection 1.00 root floor(div(cast(Column#4, decimal(20,0) BINARY), 2))->Column#6, Column#5 + └─HashAgg 1.00 root group by:Column#13, funcs:count(Column#11)->Column#5, funcs:firstrow(Column#12)->Column#4 + └─Projection 10000.00 root planner__core__casetest__integration.t1.c1->Column#11, Column#4->Column#12, floor(div(cast(Column#4, decimal(20,0) BINARY), 2))->Column#13 + └─Projection 10000.00 root setvar(rownum, plus(getvar(rownum), 1))->Column#4, planner__core__casetest__integration.t1.c1 + └─HashJoin 10000.00 root CARTESIAN inner join + ├─Projection(Build) 1.00 root setvar(rownum, -1)->Column#1 + │ └─TableDual 1.00 root rows:1 + └─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select @n:=@n+1 as e from ta group by e; +id estRows task access object operator info +Projection 1.00 root setvar(n, plus(getvar(n), 1))->Column#4 +└─HashAgg 1.00 root group by:Column#8, funcs:firstrow(1)->Column#7 + └─Projection 10000.00 root setvar(n, plus(cast(getvar(n), double BINARY), 1))->Column#8 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo +explain format = 'brief' select @n:=@n+a as e from ta group by e; +id estRows task access object operator info +Projection 8000.00 root setvar(n, plus(getvar(n), cast(planner__core__casetest__integration.ta.a, double BINARY)))->Column#4 +└─HashAgg 8000.00 root group by:Column#7, funcs:firstrow(Column#6)->planner__core__casetest__integration.ta.a + └─Projection 10000.00 root planner__core__casetest__integration.ta.a->Column#6, setvar(n, plus(getvar(n), cast(planner__core__casetest__integration.ta.a, double BINARY)))->Column#7 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo +explain format = 'brief' select * from (select @n:=@n+1 as e from ta) tt group by e; +id estRows task access object operator info +HashAgg 1.00 root group by:Column#4, funcs:firstrow(Column#4)->Column#4 +└─Projection 10000.00 root setvar(n, plus(getvar(n), 1))->Column#4 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo +explain format = 'brief' select * from (select @n:=@n+a as e from ta) tt group by e; +id estRows task access object operator info +HashAgg 8000.00 root group by:Column#4, funcs:firstrow(Column#4)->Column#4 +└─Projection 10000.00 root setvar(n, plus(getvar(n), cast(planner__core__casetest__integration.ta.a, double BINARY)))->Column#4 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo +explain format = 'brief' select a from ta group by @n:=@n+1; +id estRows task access object operator info +HashAgg 1.00 root group by:Column#5, funcs:firstrow(Column#4)->planner__core__casetest__integration.ta.a +└─Projection 10000.00 root planner__core__casetest__integration.ta.a->Column#4, setvar(n, plus(getvar(n), 1))->Column#5 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo +explain format = 'brief' select a from ta group by @n:=@n+a; +id estRows task access object operator info +HashAgg 8000.00 root group by:Column#5, funcs:firstrow(Column#4)->planner__core__casetest__integration.ta.a +└─Projection 10000.00 root planner__core__casetest__integration.ta.a->Column#4, setvar(n, plus(getvar(n), cast(planner__core__casetest__integration.ta.a, double BINARY)))->Column#5 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo +set sql_mode=default; +drop table if exists t1, t2, t3, t4, t5, t6, t7, t8; +create table t1 (a bigint key); +create table t2 (a int key); +create definer=`root`@`127.0.0.1` view v1 as (select a from t1) union (select a from t2); +create table t3 (a varchar(100) key); +create table t4 (a varchar(10) key); +create definer=`root`@`127.0.0.1` view v2 as (select a from t3) union (select a from t4); +create table t5 (a char(100) key); +create table t6 (a char(10) key); +create definer=`root`@`127.0.0.1` view v3 as (select a from t5) union (select a from t6); +create table t7 (a varchar(100) key); +create table t8 (a int key); +create definer=`root`@`127.0.0.1` view v4 as (select a from t7) union (select a from t8); +explain format='brief' select * from v1 where a = 1; -- the condition should be downcast through both side and go get point; +id estRows task access object operator info +HashAgg 2.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 2.00 root + ├─Point_Get 1.00 root table:t1 handle:1 + └─Projection 1.00 root cast(planner__core__casetest__integration.t2.a, bigint(20) BINARY)->Column#3 + └─Point_Get 1.00 root table:t2 handle:1 +select * from v1 where a = 1; -- the condition should be downcast through both side and go get point; +a +explain format='brief' select * from v1 where a = '1test'; -- the condition should be downcast through both side and go get point too; +id estRows task access object operator info +HashAgg 2.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 2.00 root + ├─Point_Get 1.00 root table:t1 handle:1 + └─Projection 1.00 root cast(planner__core__casetest__integration.t2.a, bigint(20) BINARY)->Column#3 + └─Point_Get 1.00 root table:t2 handle:1 +select * from v1 where a = '1test'; -- the condition should be downcast through both side and go get point too; +a +explain format='brief' select * from v1 where a > 1; -- the condition should be downcast through both side and go range scan; +id estRows task access object operator info +HashAgg 5333.33 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 6666.67 root + ├─TableReader 3333.33 root data:TableRangeScan + │ └─TableRangeScan 3333.33 cop[tikv] table:t1 range:(1,+inf], keep order:false, stats:pseudo + └─Projection 3333.33 root cast(planner__core__casetest__integration.t2.a, bigint(20) BINARY)->Column#3 + └─TableReader 3333.33 root data:TableRangeScan + └─TableRangeScan 3333.33 cop[tikv] table:t2 range:(1,+inf], keep order:false, stats:pseudo +select * from v1 where a > 1; -- the condition should be downcast through both side and go range scan; +a +explain format='brief' select * from v2 where a = 'test'; +id estRows task access object operator info +HashAgg 16.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 20.00 root + ├─Point_Get 1.00 root table:t3, clustered index:PRIMARY(a) + └─Projection 10.00 root cast(planner__core__casetest__integration.t4.a, varchar(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3 + └─Point_Get 1.00 root table:t4, clustered index:PRIMARY(a) +select * from v2 where a = 'test'; +a +explain format='brief' select * from v2 where a = 1; +id estRows task access object operator info +HashAgg 12800.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 16000.00 root + ├─TableReader 8000.00 root data:Selection + │ └─Selection 8000.00 cop[tikv] eq(cast(planner__core__casetest__integration.t3.a, double BINARY), 1) + │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo + └─Projection 8000.00 root cast(planner__core__casetest__integration.t4.a, varchar(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3 + └─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] eq(cast(cast(planner__core__casetest__integration.t4.a, varchar(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), double BINARY), 1) + └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo +select * from v2 where a = 1; +a +explain format='brief' select * from v2 where a > 'test'; +id estRows task access object operator info +HashAgg 5333.33 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 6666.67 root + ├─TableReader 3333.33 root data:TableRangeScan + │ └─TableRangeScan 3333.33 cop[tikv] table:t3 range:("test",+inf], keep order:false, stats:pseudo + └─Projection 3333.33 root cast(planner__core__casetest__integration.t4.a, varchar(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3 + └─TableReader 3333.33 root data:TableRangeScan + └─TableRangeScan 3333.33 cop[tikv] table:t4 range:("test",+inf], keep order:false, stats:pseudo +select * from v2 where a > 'test'; +a +explain format='brief' select * from v3 where a = 'test' -- the condition shouldn't be downcast through both side and go get point; +id estRows task access object operator info +HashAgg 6408.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 8010.00 root + ├─Point_Get 1.00 root table:t5, clustered index:PRIMARY(a) + └─Projection 8000.00 root cast(planner__core__casetest__integration.t6.a, char(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3 + └─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] eq(cast(planner__core__casetest__integration.t6.a, char(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), "test") + └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo +select * from v3 where a = 'test' -- the condition shouldn't be downcast through both side and go get point; +a +explain format='brief' select * from v3 where a > 'test' -- the condition shouldn't be downcast through both side and go get point too; +id estRows task access object operator info +HashAgg 9066.67 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 11333.33 root + ├─TableReader 3333.33 root data:TableRangeScan + │ └─TableRangeScan 3333.33 cop[tikv] table:t5 range:("test",+inf], keep order:false, stats:pseudo + └─Projection 8000.00 root cast(planner__core__casetest__integration.t6.a, char(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3 + └─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] gt(cast(planner__core__casetest__integration.t6.a, char(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), "test") + └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo +select * from v3 where a > 'test' -- the condition shouldn't be downcast through both side and go get point too; +a +explain format='brief' select * from v4 where a = 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range; +id estRows task access object operator info +HashAgg 6408.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 8010.00 root + ├─Point_Get 1.00 root table:t7, clustered index:PRIMARY(a) + └─Selection 8000.00 root eq(Column#3, "test") + └─Projection 10000.00 root cast(planner__core__casetest__integration.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo +select * from v4 where a = 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range; +a +explain format='brief' select * from v4 where a > 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range; +id estRows task access object operator info +HashAgg 9066.67 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 11333.33 root + ├─TableReader 3333.33 root data:TableRangeScan + │ └─TableRangeScan 3333.33 cop[tikv] table:t7 range:("test",+inf], keep order:false, stats:pseudo + └─Selection 8000.00 root gt(Column#3, "test") + └─Projection 10000.00 root cast(planner__core__casetest__integration.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo +select * from v4 where a > 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range; +a +set tidb_cost_model_version=2; +drop table if exists t; +create table t (id int, value decimal(10,5)); +desc format = 'brief' select count(*) from t join (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 1) v on v.id = t.id and v.v1 = t.value; +id estRows task access object operator info +StreamAgg 1.00 root funcs:count(1)->Column#10 +└─HashJoin 1.00 root inner join, equal:[eq(planner__core__casetest__integration.t.id, planner__core__casetest__integration.t.id) eq(planner__core__casetest__integration.t.value, planner__core__casetest__integration.t.value)] + ├─Selection(Build) 0.80 root not(isnull(planner__core__casetest__integration.t.id)), not(isnull(planner__core__casetest__integration.t.value)) + │ └─TopN 1.00 root planner__core__casetest__integration.t.value, offset:0, count:1 + │ └─HashJoin 12487.50 root inner join, equal:[eq(planner__core__casetest__integration.t.id, planner__core__casetest__integration.t.id)] + │ ├─TableReader(Build) 9990.00 root data:Selection + │ │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.id)) + │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo + │ └─TableReader(Probe) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.id)) + │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo + └─TableReader(Probe) 9980.01 root data:Selection + └─Selection 9980.01 cop[tikv] not(isnull(planner__core__casetest__integration.t.id)), not(isnull(planner__core__casetest__integration.t.value)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select count(*) from t join (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 1) v on v.id = t.id and v.v1 = t.value; +id estRows task access object operator info +StreamAgg 1.00 root funcs:count(1)->Column#10 +└─HashJoin 1.00 root inner join, equal:[eq(planner__core__casetest__integration.t.id, planner__core__casetest__integration.t.id) eq(planner__core__casetest__integration.t.value, planner__core__casetest__integration.t.value)] + ├─Selection(Build) 0.80 root not(isnull(planner__core__casetest__integration.t.id)), not(isnull(planner__core__casetest__integration.t.value)) + │ └─TopN 1.00 root planner__core__casetest__integration.t.value, offset:0, count:1 + │ └─HashJoin 12487.50 root inner join, equal:[eq(planner__core__casetest__integration.t.id, planner__core__casetest__integration.t.id)] + │ ├─TableReader(Build) 9990.00 root data:Selection + │ │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.id)) + │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo + │ └─TableReader(Probe) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.id)) + │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo + └─TableReader(Probe) 9980.01 root data:Selection + └─Selection 9980.01 cop[tikv] not(isnull(planner__core__casetest__integration.t.id)), not(isnull(planner__core__casetest__integration.t.value)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +set tidb_partition_prune_mode='dynamic'; +drop table if exists t1, t2_part; +create table t1 (a int, b int); +create table t2_part (a int, b int, key(a)) partition by hash(a) partitions 4; +explain select /*+ TIDB_INLJ(t2_part@sel_2) */ * from t1 where t1.b<10 and not exists (select 1 from t2_part where t1.a=t2_part.a and t2_part.b<20); +id estRows task access object operator info +HashJoin_19 2658.67 root anti semi join, equal:[eq(planner__core__casetest__integration.t1.a, planner__core__casetest__integration.t2_part.a)] +├─PartitionUnion_23(Build) 13293.33 root +│ ├─Projection_24 3323.33 root planner__core__casetest__integration.t2_part.a +│ │ └─TableReader_27 3323.33 root data:Selection_26 +│ │ └─Selection_26 3323.33 cop[tikv] lt(planner__core__casetest__integration.t2_part.b, 20) +│ │ └─TableFullScan_25 10000.00 cop[tikv] table:t2_part, partition:p0 keep order:false, stats:pseudo +│ ├─Projection_28 3323.33 root planner__core__casetest__integration.t2_part.a +│ │ └─TableReader_31 3323.33 root data:Selection_30 +│ │ └─Selection_30 3323.33 cop[tikv] lt(planner__core__casetest__integration.t2_part.b, 20) +│ │ └─TableFullScan_29 10000.00 cop[tikv] table:t2_part, partition:p1 keep order:false, stats:pseudo +│ ├─Projection_32 3323.33 root planner__core__casetest__integration.t2_part.a +│ │ └─TableReader_35 3323.33 root data:Selection_34 +│ │ └─Selection_34 3323.33 cop[tikv] lt(planner__core__casetest__integration.t2_part.b, 20) +│ │ └─TableFullScan_33 10000.00 cop[tikv] table:t2_part, partition:p2 keep order:false, stats:pseudo +│ └─Projection_36 3323.33 root planner__core__casetest__integration.t2_part.a +│ └─TableReader_39 3323.33 root data:Selection_38 +│ └─Selection_38 3323.33 cop[tikv] lt(planner__core__casetest__integration.t2_part.b, 20) +│ └─TableFullScan_37 10000.00 cop[tikv] table:t2_part, partition:p3 keep order:false, stats:pseudo +└─TableReader_22(Probe) 3323.33 root data:Selection_21 + └─Selection_21 3323.33 cop[tikv] lt(planner__core__casetest__integration.t1.b, 10) + └─TableFullScan_20 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +Level Code Message +Warning 1105 disable dynamic pruning due to t2_part has no global stats +Warning 1815 Optimizer Hint /*+ INL_JOIN(t2_part) */ or /*+ TIDB_INLJ(t2_part) */ is inapplicable +set @@tidb_opt_fix_control = "44262:ON"; +explain select /*+ TIDB_INLJ(t2_part@sel_2) */ * from t1 where t1.b<10 and not exists (select 1 from t2_part where t1.a=t2_part.a and t2_part.b<20); +id estRows task access object operator info +IndexJoin_13 2658.67 root anti semi join, inner:IndexLookUp_12, outer key:planner__core__casetest__integration.t1.a, inner key:planner__core__casetest__integration.t2_part.a, equal cond:eq(planner__core__casetest__integration.t1.a, planner__core__casetest__integration.t2_part.a) +├─TableReader_18(Build) 3323.33 root data:Selection_17 +│ └─Selection_17 3323.33 cop[tikv] lt(planner__core__casetest__integration.t1.b, 10) +│ └─TableFullScan_16 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─IndexLookUp_12(Probe) 4154.17 root partition:all + ├─IndexRangeScan_9(Build) 12500.00 cop[tikv] table:t2_part, index:a(a) range: decided by [eq(planner__core__casetest__integration.t2_part.a, planner__core__casetest__integration.t1.a)], keep order:false, stats:pseudo + └─Selection_11(Probe) 4154.17 cop[tikv] lt(planner__core__casetest__integration.t2_part.b, 20) + └─TableRowIDScan_10 12500.00 cop[tikv] table:t2_part keep order:false, stats:pseudo +drop table if exists t; +create table t(a int(11) not null, b int) partition by range (a) (partition p0 values less than (4), partition p1 values less than(10), partition p2 values less than maxvalue); +insert into t values (1, 1),(10, 10),(11, 11); +set tidb_opt_fix_control='44262:ON'; +explain format = 'brief' select * from t where a in (1, 2,'11'); +id estRows task access object operator info +TableReader 30.00 root partition:p0,p2 data:Selection +└─Selection 30.00 cop[tikv] in(planner__core__casetest__integration.t.a, 1, 2, 11) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where a in (17, null); +id estRows task access object operator info +TableReader 10.00 root partition:p0,p2 data:Selection +└─Selection 10.00 cop[tikv] in(planner__core__casetest__integration.t.a, 17, NULL) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where a in (16, 'abc'); +id estRows task access object operator info +TableReader 20.00 root partition:p0,p2 data:Selection +└─Selection 20.00 cop[tikv] in(planner__core__casetest__integration.t.a, 16, 0) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where a in (15, 0.12, 3.47); +id estRows task access object operator info +TableReader 10.00 root partition:p2 data:Selection +└─Selection 10.00 cop[tikv] or(eq(planner__core__casetest__integration.t.a, 15), 0) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where a in (0.12, 3.47); +id estRows task access object operator info +TableDual 0.00 root rows:0 +explain format = 'brief' select * from t where a in (14, floor(3.47)); +id estRows task access object operator info +TableReader 20.00 root partition:p0,p2 data:Selection +└─Selection 20.00 cop[tikv] in(planner__core__casetest__integration.t.a, 14, 3) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b in (3, 4); +id estRows task access object operator info +TableReader 20.00 root partition:all data:Selection +└─Selection 20.00 cop[tikv] in(planner__core__casetest__integration.t.b, 3, 4) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +set tidb_opt_fix_control=default; +drop table if exists pt; +create table pt (id int, c int, key i_id(id), key i_c(c)) partition by range (c) ( +partition p0 values less than (4), +partition p1 values less than (7), +partition p2 values less than (10)); +set @@tidb_enable_index_merge = 1; +set tidb_opt_fix_control='44262:ON'; +## Table reader +explain format='brief' select * from pt where c > 10; +id estRows task access object operator info +TableReader 3333.33 root partition:dual data:Selection +└─Selection 3333.33 cop[tikv] gt(planner__core__casetest__integration.pt.c, 10) + └─TableFullScan 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo +explain format='brief' select * from pt where c > 8; +id estRows task access object operator info +TableReader 3333.33 root partition:p2 data:Selection +└─Selection 3333.33 cop[tikv] gt(planner__core__casetest__integration.pt.c, 8) + └─TableFullScan 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo +explain format='brief' select * from pt where c < 2 or c >= 9; +id estRows task access object operator info +TableReader 6656.67 root partition:p0,p2 data:Selection +└─Selection 6656.67 cop[tikv] or(lt(planner__core__casetest__integration.pt.c, 2), ge(planner__core__casetest__integration.pt.c, 9)) + └─TableFullScan 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo +## Index reader +explain format='brief' select c from pt; +id estRows task access object operator info +IndexReader 10000.00 root partition:all index:IndexFullScan +└─IndexFullScan 10000.00 cop[tikv] table:pt, index:i_c(c) keep order:false, stats:pseudo +explain format='brief' select c from pt where c > 10; +id estRows task access object operator info +IndexReader 3333.33 root partition:dual index:IndexRangeScan +└─IndexRangeScan 3333.33 cop[tikv] table:pt, index:i_c(c) range:(10,+inf], keep order:false, stats:pseudo +explain format='brief' select c from pt where c > 8; +id estRows task access object operator info +IndexReader 3333.33 root partition:p2 index:IndexRangeScan +└─IndexRangeScan 3333.33 cop[tikv] table:pt, index:i_c(c) range:(8,+inf], keep order:false, stats:pseudo +explain format='brief' select c from pt where c < 2 or c >= 9; +id estRows task access object operator info +IndexReader 6656.67 root partition:p0,p2 index:IndexRangeScan +└─IndexRangeScan 6656.67 cop[tikv] table:pt, index:i_c(c) range:[-inf,2), [9,+inf], keep order:false, stats:pseudo +## Index Lookup +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt; +id estRows task access object operator info +IndexLookUp 10000.00 root partition:all +├─IndexFullScan(Build) 10000.00 cop[tikv] table:pt, index:i_id(id) keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt where id < 4 and c > 10; +id estRows task access object operator info +IndexLookUp 1107.78 root partition:dual +├─IndexRangeScan(Build) 3323.33 cop[tikv] table:pt, index:i_id(id) range:[-inf,4), keep order:false, stats:pseudo +└─Selection(Probe) 1107.78 cop[tikv] gt(planner__core__casetest__integration.pt.c, 10) + └─TableRowIDScan 3323.33 cop[tikv] table:pt keep order:false, stats:pseudo +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c > 8; +id estRows task access object operator info +IndexLookUp 1107.78 root partition:p2 +├─IndexRangeScan(Build) 3323.33 cop[tikv] table:pt, index:i_id(id) range:[-inf,10), keep order:false, stats:pseudo +└─Selection(Probe) 1107.78 cop[tikv] gt(planner__core__casetest__integration.pt.c, 8) + └─TableRowIDScan 3323.33 cop[tikv] table:pt keep order:false, stats:pseudo +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c < 2 or c >= 9; +id estRows task access object operator info +IndexLookUp 5325.33 root partition:p0,p2 +├─IndexFullScan(Build) 10000.00 cop[tikv] table:pt, index:i_id(id) keep order:false, stats:pseudo +└─Selection(Probe) 5325.33 cop[tikv] or(and(lt(planner__core__casetest__integration.pt.id, 10), lt(planner__core__casetest__integration.pt.c, 2)), ge(planner__core__casetest__integration.pt.c, 9)) + └─TableRowIDScan 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo +## Partition selection +explain format='brief' select * from pt partition (p0) where c > 8; +id estRows task access object operator info +TableReader 3333.33 root partition:dual data:Selection +└─Selection 3333.33 cop[tikv] gt(planner__core__casetest__integration.pt.c, 8) + └─TableFullScan 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo +explain format='brief' select c from pt partition (p0, p2) where c > 8; +id estRows task access object operator info +IndexReader 3333.33 root partition:p2 index:IndexRangeScan +└─IndexRangeScan 3333.33 cop[tikv] table:pt, index:i_c(c) range:(8,+inf], keep order:false, stats:pseudo +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt partition (p1, p2) where c < 3 and id = 5; +id estRows task access object operator info +IndexLookUp 3.32 root partition:dual +├─IndexRangeScan(Build) 10.00 cop[tikv] table:pt, index:i_id(id) range:[5,5], keep order:false, stats:pseudo +└─Selection(Probe) 3.32 cop[tikv] lt(planner__core__casetest__integration.pt.c, 3) + └─TableRowIDScan 10.00 cop[tikv] table:pt keep order:false, stats:pseudo +## Index Merge +explain format='brief' select * from pt where id = 4 or c < 7; +id estRows task access object operator info +IndexMerge 3330.01 root partition:all type: union +├─IndexRangeScan(Build) 10.00 cop[tikv] table:pt, index:i_id(id) range:[4,4], keep order:false, stats:pseudo +├─IndexRangeScan(Build) 3323.33 cop[tikv] table:pt, index:i_c(c) range:[-inf,7), keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 3330.01 cop[tikv] table:pt keep order:false, stats:pseudo +explain format='brief' select * from pt where id > 4 or c = 7; +id estRows task access object operator info +IndexMerge 3340.00 root partition:all type: union +├─IndexRangeScan(Build) 3333.33 cop[tikv] table:pt, index:i_id(id) range:(4,+inf], keep order:false, stats:pseudo +├─IndexRangeScan(Build) 10.00 cop[tikv] table:pt, index:i_c(c) range:[7,7], keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 3340.00 cop[tikv] table:pt keep order:false, stats:pseudo +set tidb_opt_fix_control=default; +set @@tidb_enable_index_merge = default; +drop table if exists github_events; +CREATE TABLE `github_events` ( +`id` bigint(20) NOT NULL DEFAULT '0', +`type` varchar(29) NOT NULL DEFAULT 'Event', +`created_at` datetime NOT NULL DEFAULT '1970-01-01 00:00:00', +`repo_id` bigint(20) NOT NULL DEFAULT '0', +`repo_name` varchar(140) NOT NULL DEFAULT '', +`actor_id` bigint(20) NOT NULL DEFAULT '0', +`actor_login` varchar(40) NOT NULL DEFAULT '', +`language` varchar(26) NOT NULL DEFAULT '', +`additions` bigint(20) NOT NULL DEFAULT '0', +`deletions` bigint(20) NOT NULL DEFAULT '0', +`action` varchar(11) NOT NULL DEFAULT '', +`number` int(11) NOT NULL DEFAULT '0', +`commit_id` varchar(40) NOT NULL DEFAULT '', +`comment_id` bigint(20) NOT NULL DEFAULT '0', +`org_login` varchar(40) NOT NULL DEFAULT '', +`org_id` bigint(20) NOT NULL DEFAULT '0', +`state` varchar(6) NOT NULL DEFAULT '', +`closed_at` datetime NOT NULL DEFAULT '1970-01-01 00:00:00', +`comments` int(11) NOT NULL DEFAULT '0', +`pr_merged_at` datetime NOT NULL DEFAULT '1970-01-01 00:00:00', +`pr_merged` tinyint(1) NOT NULL DEFAULT '0', +`pr_changed_files` int(11) NOT NULL DEFAULT '0', +`pr_review_comments` int(11) NOT NULL DEFAULT '0', +`pr_or_issue_id` bigint(20) NOT NULL DEFAULT '0', +`event_day` date NOT NULL, +`event_month` date NOT NULL, +`event_year` int(11) NOT NULL, +`push_size` int(11) NOT NULL DEFAULT '0', +`push_distinct_size` int(11) NOT NULL DEFAULT '0', +`creator_user_login` varchar(40) NOT NULL DEFAULT '', +`creator_user_id` bigint(20) NOT NULL DEFAULT '0', +`pr_or_issue_created_at` datetime NOT NULL DEFAULT '1970-01-01 00:00:00', +KEY `index_github_events_on_id` (`id`), +KEY `index_github_events_on_created_at` (`created_at`), +KEY `index_github_events_on_repo_id_type_action_month_actor_login` (`repo_id`,`type`,`action`,`event_month`,`actor_login`), +KEY `index_ge_on_repo_id_type_action_pr_merged_created_at_add_del` (`repo_id`,`type`,`action`,`pr_merged`,`created_at`,`additions`,`deletions`), +KEY `index_ge_on_creator_id_type_action_merged_created_at_add_del` (`creator_user_id`,`type`,`action`,`pr_merged`,`created_at`,`additions`,`deletions`), +KEY `index_ge_on_actor_id_type_action_created_at_repo_id_commits` (`actor_id`,`type`,`action`,`created_at`,`repo_id`,`push_distinct_size`), +KEY `index_ge_on_repo_id_type_action_created_at_number_pdsize_psize` (`repo_id`,`type`,`action`,`created_at`,`number`,`push_distinct_size`,`push_size`), +KEY `index_ge_on_repo_id_type_action_created_at_actor_login` (`repo_id`,`type`,`action`,`created_at`,`actor_login`), +KEY `index_ge_on_repo_name_type` (`repo_name`,`type`), +KEY `index_ge_on_actor_login_type` (`actor_login`,`type`), +KEY `index_ge_on_org_login_type` (`org_login`,`type`), +KEY `index_ge_on_language` (`language`), +KEY `index_ge_on_org_id_type` (`org_id`,`type`), +KEY `index_ge_on_actor_login_lower` ((lower(`actor_login`))), +KEY `index_ge_on_repo_name_lower` ((lower(`repo_name`))), +KEY `index_ge_on_language_lower` ((lower(`language`))), +KEY `index_ge_on_type_action` (`type`,`action`) /*!80000 INVISIBLE */, +KEY `index_ge_on_repo_id_type_created_at` (`repo_id`,`type`,`created_at`), +KEY `index_ge_on_repo_id_created_at` (`repo_id`,`created_at`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY LIST COLUMNS(`type`) +(PARTITION `push_event` VALUES IN ('PushEvent'), +PARTITION `create_event` VALUES IN ('CreateEvent'), +PARTITION `pull_request_event` VALUES IN ('PullRequestEvent'), +PARTITION `watch_event` VALUES IN ('WatchEvent'), +PARTITION `issue_comment_event` VALUES IN ('IssueCommentEvent'), +PARTITION `issues_event` VALUES IN ('IssuesEvent'), +PARTITION `delete_event` VALUES IN ('DeleteEvent'), +PARTITION `fork_event` VALUES IN ('ForkEvent'), +PARTITION `pull_request_review_comment_event` VALUES IN ('PullRequestReviewCommentEvent'), +PARTITION `pull_request_review_event` VALUES IN ('PullRequestReviewEvent'), +PARTITION `gollum_event` VALUES IN ('GollumEvent'), +PARTITION `release_event` VALUES IN ('ReleaseEvent'), +PARTITION `member_event` VALUES IN ('MemberEvent'), +PARTITION `commit_comment_event` VALUES IN ('CommitCommentEvent'), +PARTITION `public_event` VALUES IN ('PublicEvent'), +PARTITION `gist_event` VALUES IN ('GistEvent'), +PARTITION `follow_event` VALUES IN ('FollowEvent'), +PARTITION `event` VALUES IN ('Event'), +PARTITION `download_event` VALUES IN ('DownloadEvent'), +PARTITION `team_add_event` VALUES IN ('TeamAddEvent'), +PARTITION `fork_apply_event` VALUES IN ('ForkApplyEvent')); +SELECT +repo_id, GROUP_CONCAT( +DISTINCT actor_login +ORDER BY cnt DESC +SEPARATOR ',' +) AS actor_logins +FROM ( +SELECT +ge.repo_id AS repo_id, +ge.actor_login AS actor_login, +COUNT(*) AS cnt +FROM github_events ge +WHERE +type = 'PullRequestEvent' AND action = 'opened' +AND (ge.created_at >= DATE_SUB(NOW(), INTERVAL 1 DAY) AND ge.created_at <= NOW()) +GROUP BY ge.repo_id, ge.actor_login +ORDER BY cnt DESC +) sub +GROUP BY repo_id; +repo_id actor_logins +drop database if exists testdb; +create database testdb; +use testdb; +drop table if exists `t270`; +CREATE TABLE `t270` ( +`vkey` int(11) DEFAULT NULL, +`pkey` int(11) DEFAULT NULL, +`c1128` varchar(100) DEFAULT NULL, +`c1129` int(11) DEFAULT NULL, +`c1130` varchar(100) DEFAULT NULL, +`c1131` double DEFAULT NULL, +`c1132` varchar(100) DEFAULT NULL, +`c1133` double DEFAULT NULL, +`c1134` varchar(100) DEFAULT NULL, +`c1135` int(11) DEFAULT NULL +); +drop table if exists `t271`; +CREATE TABLE `t271` ( +`vkey` int(11) DEFAULT NULL, +`pkey` int(11) DEFAULT NULL, +`c1136` varchar(100) DEFAULT NULL, +`c1137` int(11) DEFAULT NULL, +`c1138` varchar(100) DEFAULT NULL, +`c1139` int(11) DEFAULT NULL, +`c1140` double DEFAULT NULL, +`c1141` int(11) DEFAULT NULL +); +drop table if exists `t272`; +CREATE TABLE `t272` ( +`vkey` int(11) DEFAULT NULL, +`pkey` int(11) DEFAULT NULL, +`c1142` int(11) DEFAULT NULL, +`c1143` varchar(100) DEFAULT NULL, +`c1144` int(11) DEFAULT NULL, +`c1145` int(11) DEFAULT NULL, +`c1146` varchar(100) DEFAULT NULL, +`c1147` double DEFAULT NULL, +`c1148` varchar(100) DEFAULT NULL, +`c1149` double DEFAULT NULL +); +CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`%` SQL SECURITY DEFINER VIEW `t273_test` (`c0`, `c1`, `c2`, `c3`, `c4`) AS SELECT AVG(37) OVER (PARTITION BY `ref_0`.`c1136` ORDER BY `ref_0`.`vkey` DESC,`ref_0`.`pkey` DESC,`ref_0`.`c1136` DESC,`ref_0`.`c1137`,`ref_0`.`c1138` DESC,`ref_0`.`c1139` DESC,`ref_0`.`c1140` DESC,`ref_0`.`c1141`) AS `c0`,COALESCE(`ref_0`.`c1137`, `ref_0`.`c1141`) AS `c1`,`ref_0`.`vkey` AS `c2`,`ref_0`.`pkey` AS `c3`,`ref_0`.`c1138` AS `c4` FROM `testdb`.`t271` AS `ref_0` WHERE EXISTS (SELECT `subq_0`.`c2` AS `c0`,`subq_0`.`c0` AS `c1`,`subq_0`.`c0` AS `c2`,`subq_0`.`c0` AS `c3`,CASE WHEN EXISTS (SELECT `ref_9`.`c1131` AS `c0`,`ref_9`.`c1131` AS `c1`,(FALSE) XOR (((-45)=(-69)) OR ((-0)>(-71))) AS `c2`,`ref_9`.`c1133` AS `c3`,`ref_9`.`c1128` AS `c4`,-0 AS `c5`,1 AS `c6`,`ref_9`.`c1132` AS `c7`,`ref_9`.`c1131` AS `c8`,`ref_9`.`c1130` AS `c9`,NULL AS `c10` FROM `testdb`.`t270` AS `ref_9` WHERE (-0)<(-8) UNION ALL SELECT `ref_0`.`c1140` AS `c0`,`ref_11`.`c1133` AS `c1`,(NULL)<(NULL) AS `c2`,`ref_0`.`c1140` AS `c3`,`ref_0`.`c1136` AS `c4`,95 AS `c5`,NOT (_UTF8MB4'mum#M' LIKE _UTF8MB4'%_U') AS `c6`,`ref_11`.`c1128` AS `c7`,`ref_11`.`c1131` AS `c8`,(SELECT `c1143` AS `c1143` FROM `testdb`.`t272` ORDER BY `c1143` LIMIT 3,1) AS `c9`,97 AS `c10` FROM `testdb`.`t270` AS `ref_11` WHERE NOT (TRUE)) THEN _UTF8MB4'xf' ELSE _UTF8MB4'>c' END LIKE _UTF8MB4'_^^' AS `c4`,`subq_0`.`c1` AS `c5`,`ref_0`.`vkey` AS `c6`,((`subq_0`.`c1`)=(SELECT `ref_12`.`c1132` AS `c0` FROM `testdb`.`t270` AS `ref_12` WHERE TRUE ORDER BY `c0` DESC LIMIT 1)) XOR ((`ref_0`.`pkey`)>=(SELECT (SELECT `vkey` AS `vkey` FROM `testdb`.`t271` ORDER BY `vkey` LIMIT 1,1) AS `c0` FROM `testdb`.`t271` AS `ref_13` WHERE (-24)<=((SELECT COUNT(`c1140`) AS `count(c1140)` FROM `testdb`.`t271`)) ORDER BY `c0` LIMIT 1)) AS `c7`,`ref_0`.`pkey` AS `c8`,`subq_0`.`c2` AS `c9`,`ref_0`.`vkey` AS `c10`,`ref_0`.`c1139` AS `c11`,TRUE AS `c12`,`subq_0`.`c0` AS `c13`,`subq_0`.`c2` AS `c14`,`subq_0`.`c2` AS `c15`,FALSE AS `c16`,CASE WHEN ((FALSE) OR ((((FALSE) XOR (((-73)<(-91)) OR (((-0) BETWEEN (-0) AND (-0)) AND ((NULL) OR ((0)>((SELECT COUNT(`c1131`) AS `count(c1131)` FROM `testdb`.`t270`))))))) AND ((-19)>(NULL))) OR (((77)<(73)) AND (NOT (((73) IN (SELECT 0 AS `c0` FROM `testdb`.`t271` AS `ref_14` WHERE (NULL) AND (NULL) EXCEPT SELECT NULL AS `c0` FROM `testdb`.`t270` AS `ref_15` WHERE (`ref_15`.`c1131`)!=(SELECT `ref_15`.`c1133` AS `c0` FROM `testdb`.`t270` AS `ref_16` WHERE _UTF8MB4'$@-X' LIKE _UTF8MB4'__%' ORDER BY `c0` DESC LIMIT 1))) IS TRUE))))) OR (NOT ((-24)<=(-43))) THEN `subq_0`.`c1` ELSE `subq_0`.`c2` END AS `c17`,`subq_0`.`c1` AS `c18`,`subq_0`.`c0` AS `c19`,`subq_0`.`c0` AS `c20`,`subq_0`.`c2` AS `c21`,`subq_0`.`c0` AS `c22`,`subq_0`.`c2` AS `c23`,`subq_0`.`c0` AS `c24`,`ref_0`.`c1141` AS `c25` FROM (SELECT DISTINCT TRUE AS `c0`,`ref_1`.`c1143` AS `c1`,`ref_1`.`c1146` AS `c2` FROM `testdb`.`t272` AS `ref_1` WHERE NOT (((`ref_0`.`c1136`)!=(SELECT `ref_2`.`c1146` AS `c0` FROM `testdb`.`t272` AS `ref_2` WHERE (62) BETWEEN ((SELECT COUNT(`c1147`) AS `count(c1147)` FROM `testdb`.`t272`)) AND (-0) ORDER BY `c0` LIMIT 1)) XOR ((-0) BETWEEN (0) AND (-0)))) AS `subq_0` WHERE (CHAR_LENGTH(CASE WHEN ((`subq_0`.`c0`) IS NOT NULL) OR ((`ref_0`.`c1138`)>(SELECT `ref_0`.`c1138` AS `c0` FROM `testdb`.`t272` AS `ref_3` WHERE FALSE ORDER BY `c0` DESC LIMIT 1)) THEN _UTF8MB4'' ELSE _UTF8MB4'tL' END)) BETWEEN (ABS(46%-11)) AND (CASE WHEN (((((`subq_0`.`c2`) IN (SELECT `ref_4`.`c1134` AS `c0` FROM `testdb`.`t270` AS `ref_4` WHERE (NULL LIKE _UTF8MB4'%Ny') OR (EXISTS (SELECT DISTINCT `ref_5`.`c1136` AS `c0`,`ref_5`.`c1140` AS `c1` FROM `testdb`.`t271` AS `ref_5` WHERE FALSE UNION ALL SELECT `ref_4`.`c1130` AS `c0`,`ref_4`.`c1131` AS `c1` FROM `testdb`.`t271` AS `ref_6` WHERE (-97) BETWEEN (73) AND (-10))) UNION ALL SELECT `ref_7`.`c1138` AS `c0` FROM `testdb`.`t271` AS `ref_7` WHERE FALSE)) IS TRUE) OR (NULL)) AND ((NULL)>=((SELECT COUNT(`c1140`) AS `count(c1140)` FROM `testdb`.`t271`)))) XOR (((`ref_0`.`vkey`) IN (SELECT `ref_8`.`c1145` AS `c0` FROM `testdb`.`t272` AS `ref_8` WHERE ((FALSE) AND (NULL)) OR ((`ref_8`.`c1144`) IS NULL))) IS TRUE) THEN 87 ELSE CASE WHEN ((`ref_0`.`c1138`) IS NULL) OR ((-22)!=(-0)) THEN 17 ELSE -67 END END)) ORDER BY `c0` DESC,`c1` DESC,`c2`,`c3`,`c4` DESC; +select +(select +subq_1.c0 as c0 +from +t273_test as ref_84 +where exists ( +select +(select +ref_86.c1147 as c0 +from +t272 as ref_86 +where (subq_1.c0) > (subq_1.c0) +window w0 as (partition by ref_86.c1147 order by ref_86.c1143 desc) +order by c0 limit 1 +) as c3, +(select +subq_1.c0 as c0 +from +t273_test as ref_89 +order by c0 limit 1) as c4 +from +t271 as ref_85 +) +order by c0 desc limit 1) as c1 +from +(select 1 as c0) as subq_1; +c1 +NULL +select +(select +subq_1.c0 as c0 +from +t271 as ref_84 +where exists ( +select +(select +ref_86.c1147 as c0 +from +t272 as ref_86 +where (subq_1.c0) > (subq_1.c0) +window w0 as (partition by ref_86.c1147 order by ref_86.c1143 desc) +order by c0 limit 1 +) as c3, +(select +subq_1.c0 as c0 +from +t271 as ref_89 +order by c0 limit 1) as c4 +from +t271 as ref_85 +) +order by c0 desc limit 1) as c1 +from +(select 1 as c0) as subq_1; +c1 +NULL diff --git a/tests/integrationtest/t/planner/core/casetest/integration.test b/tests/integrationtest/t/planner/core/casetest/integration.test new file mode 100644 index 0000000000000..50e2faf7536a7 --- /dev/null +++ b/tests/integrationtest/t/planner/core/casetest/integration.test @@ -0,0 +1,719 @@ +# TestAggColumnPrune +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int); +insert into t values(1),(2); +select count(1) from t join (select count(1) from t where false) as tmp; +select count(1) from t join (select max(a) from t where false) as tmp; +select count(1) from t join (select min(a) from t where false) as tmp; +select count(1) from t join (select sum(a) from t where false) as tmp; +select count(1) from t join (select avg(a) from t where false) as tmp; +select count(1) from t join (select count(1) from t where false group by a) as tmp; +select count(1) from t join (select max(a) from t where false group by a) as tmp; +select count(1) from t join (select min(a) from t where false group by a) as tmp; +select count(1) from t join (select sum(a) from t where false group by a) as tmp; +select count(1) from t join (select avg(a) from t where false group by a) as tmp; +SELECT avg(2) FROM(SELECT min(c) FROM t JOIN(SELECT 1 c) d ORDER BY a) e; + +# TestIsFromUnixtimeNullRejective +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a bigint, b bigint); +explain format = 'brief' select * from t t1 left join t t2 on t1.a=t2.a where from_unixtime(t2.b); + +# TestSimplifyOuterJoinWithCast +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int not null, b datetime default null); +explain format = 'brief' select * from t t1 left join t t2 on t1.a = t2.a where cast(t1.b as date) >= '2019-01-01'; + +# TestPartitionTableStats +set @@tidb_partition_prune_mode='static'; +set tidb_opt_limit_push_down_threshold=0; +drop table if exists t; +create table t(a int, b int)partition by range columns(a)(partition p0 values less than (10), partition p1 values less than(20), partition p2 values less than(30)); +insert into t values(21, 1), (22, 2), (23, 3), (24, 4), (15, 5); +analyze table t; +explain format = 'brief' select * from t order by a; +select * from t order by a; +explain format = 'brief' select * from t order by a limit 3; +select * from t order by a limit 3; +set tidb_opt_limit_push_down_threshold=default; + +# TestMaxMinEliminate +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int primary key); +set tidb_enable_clustered_index='ON'; +create table cluster_index_t(a int, b int, c int, primary key (a, b)); +explain format = 'brief' (select max(a) from t) union (select min(a) from t); +explain format = 'brief' select min(a), max(a) from cluster_index_t; +explain format = 'brief' select min(b), max(b) from cluster_index_t where a = 1; +explain format = 'brief' select min(a), max(a) from cluster_index_t where b = 1; +explain format = 'brief' select min(b), max(b) from cluster_index_t where b = 1; +set tidb_enable_clustered_index=DEFAULT; + +# TestSubqueryWithTopN +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int, b int); +desc format = 'brief' select t1.b from t t1 where t1.b in (select t2.a from t t2 order by t1.a+t2.a limit 1); +desc format = 'brief' select t1.a from t t1 order by (t1.b = 1 and exists (select 1 from t t2 where t1.b = t2.b)) limit 1; +desc format = 'brief' select * from (select b+b as x from t) t1, t t2 where t1.x=t2.b order by t1.x limit 1; + +# TestApproxPercentile +drop table if exists t; +create table t(a int, b int); +insert into t values(1, 1), (2, 1), (3, 2), (4, 2), (5, 2); +explain select approx_percentile(a, 50) from t; +select approx_percentile(a, 50) from t; +explain select approx_percentile(a, 10) from t; +select approx_percentile(a, 10) from t; +explain select approx_percentile(a, 10+70) from t; +select approx_percentile(a, 10+70) from t; +explain select approx_percentile(a, 10*10) from t; +select approx_percentile(a, 10*10) from t; +explain select approx_percentile(a, 50) from t group by b order by b; +select approx_percentile(a, 50) from t group by b order by b; + +# TestStreamAggProp +drop table if exists t; +create table t(a int); +insert into t values(1),(1),(2); +explain format = 'brief' select /*+ stream_agg() */ count(*) c from t group by a order by c limit 1; +select /*+ stream_agg() */ count(*) c from t group by a order by c limit 1; +explain format = 'brief' select /*+ stream_agg() */ count(*) c from t group by a order by c; +select /*+ stream_agg() */ count(*) c from t group by a order by c; +explain format = 'brief' select /*+ stream_agg() */ count(*) c from t group by a order by a limit 1; +select /*+ stream_agg() */ count(*) c from t group by a order by a limit 1; +explain format = 'brief' select /*+ stream_agg() */ count(*) c from t group by a order by a; +select /*+ stream_agg() */ count(*) c from t group by a order by a; + +# TestIssue20710 +drop table if exists t; +drop table if exists s; +create table t(a int, b int); +create table s(a int, b int, index(a)); +insert into t values(1,1),(1,2),(2,2); +insert into s values(1,1),(2,2),(2,1); +explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.b; +explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.a; +explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.a = s.b; +explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.b; +explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.a; +explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.a = s.b; + +# TestIssue23887 +drop table if exists t; +create table t(a int, b int); +insert into t values(1, 2), (3, 4); +explain format = 'brief' select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t; +select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t; +drop table if exists t1; +create table t1 (c1 int primary key, c2 int, c3 int, index c2 (c2)); +select count(1) from (select count(1) from (select * from t1 where c3 = 100) k) k2; + +# TestReorderSimplifiedOuterJoins +set tidb_cost_model_version=2; +drop table if exists t1,t2,t3; +create table t1 (pk char(32) primary key nonclustered, col1 char(32), col2 varchar(40), col3 char(32), key (col1), key (col3), key (col2,col3), key (col1,col3)); +create table t2 (pk char(32) primary key nonclustered, col1 varchar(100)); +create table t3 (pk char(32) primary key nonclustered, keycol varchar(100), pad1 tinyint(1) default null, pad2 varchar(40), key (keycol,pad1,pad2)); +explain format = 'brief' SELECT t1.pk FROM t1 INNER JOIN t2 ON t1.col1 = t2.pk INNER JOIN t3 ON t1.col3 = t3.pk WHERE t2.col1 IN ('a' , 'b') AND t3.keycol = 'c' AND t1.col2 = 'a' AND t1.col1 != 'abcdef' AND t1.col1 != 'aaaaaa'; +explain format = 'brief' SELECT t1.pk FROM t1 LEFT JOIN t2 ON t1.col1 = t2.pk LEFT JOIN t3 ON t1.col3 = t3.pk WHERE t2.col1 IN ('a' , 'b') AND t3.keycol = 'c' AND t1.col2 = 'a' AND t1.col1 != 'abcdef' AND t1.col1 != 'aaaaaa'; + +# TestIsMatchProp +drop table if exists t1, t2; +create table t1(a int, b int, c int, d int, index idx_a_b_c(a, b, c)); +create table t2(a int, b int, c int, d int, index idx_a_b_c_d(a, b, c, d)); +explain format = 'brief' select a, b, c from t1 where a > 3 and b = 4 order by a, c; +explain format = 'brief' select * from t2 where a = 1 and c = 2 order by b, d; +explain format = 'brief' select a, b, c from t1 where (a = 1 and b = 1 and c = 1) or (a = 1 and b = 1 and c = 2) order by c; +explain format = 'brief' select a, b, c from t1 where (a = 1 and b = 1 and c < 3) or (a = 1 and b = 1 and c > 6) order by c; +explain format = 'brief' select * from t2 where ((a = 1 and b = 1 and d < 3) or (a = 1 and b = 1 and d > 6)) and c = 3 order by d; + +# TestDecorrelateInnerJoinInSubquery +drop table if exists t; +create table t(a int not null, b int not null); +explain format = 'brief' select * from t where exists (select 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a); +explain format = 'brief' select * from t where exists (select 1 from t t1 join t t2 on t1.a = t2.a and t1.a = t.a); +explain format = 'brief' select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a); +explain format = 'brief' select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 on t1.a = t2.a and t1.a = t.a); +explain format = 'brief' select /*+ hash_join_build(t) */ * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a); +explain format = 'brief' select /*+ hash_join_probe(t) */ * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a); + +# TestDecorrelateLimitInSubquery +drop table if exists test; +create table test(id int, value int); +drop table if exists t; +create table t(c int); +insert t values(10), (8), (7), (9), (11); +explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id limit 1); +explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id); +explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id limit 1,2); +explain format = 'brief' select * from t where 9 in (select c from t s where s.c < t.c limit 3); + +# TestConvertRangeToPoint +drop table if exists t0; +create table t0 (a int, b int, index(a, b)); +insert into t0 values (1, 1); +insert into t0 values (2, 2); +insert into t0 values (2, 2); +insert into t0 values (2, 2); +insert into t0 values (2, 2); +insert into t0 values (2, 2); +insert into t0 values (3, 3); +drop table if exists t1; +create table t1 (a int, b int, c int, index(a, b, c)); +drop table if exists t2; +create table t2 (a float, b float, index(a, b)); +drop table if exists t3; +create table t3 (a char(10), b char(10), c char(10), index(a, b, c)); +explain format = 'brief' select * from t0 where a > 1 and a < 3 order by b limit 2; +explain format = 'brief' select * from t1 where a >= 2 and a <= 2 and b = 2 and c > 2; +explain format = 'brief' select * from t2 where a >= 2.5 and a <= 2.5 order by b limit 2; +explain format = 'brief' select * from t3 where a >= 'a' and a <= 'a' and b = 'b' and c > 'c'; + +# TestIssue22105 +drop table if exists t1; +CREATE TABLE t1 ( + key1 int(11) NOT NULL, + key2 int(11) NOT NULL, + key3 int(11) NOT NULL, + key4 int(11) NOT NULL, + key5 int(11) DEFAULT NULL, + key6 int(11) DEFAULT NULL, + key7 int(11) NOT NULL, + key8 int(11) NOT NULL, + KEY i1 (key1), + KEY i2 (key2), + KEY i3 (key3), + KEY i4 (key4), + KEY i5 (key5), + KEY i6 (key6) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +explain format = 'brief' SELECT /*+ use_index_merge(t1)*/ COUNT(*) FROM t1 WHERE (key4=42 AND key6 IS NOT NULL) OR (key1=4 AND key3=6); + +# TestRegardNULLAsPoint +drop table if exists tpk; +create table tuk (a int, b int, c int, unique key (a, b, c)); +create table tik (a int, b int, c int, key (a, b, c)); +insert into tuk values (NULL, NULL, NULL); +insert into tik values (NULL, NULL, NULL); +insert into tuk values (NULL, NULL, NULL); +insert into tik values (NULL, NULL, NULL); +insert into tuk values (NULL, NULL, 1); +insert into tik values (NULL, NULL, 1); +insert into tuk values (NULL, NULL, 1); +insert into tik values (NULL, NULL, 1); +insert into tuk values (NULL, 1, NULL); +insert into tik values (NULL, 1, NULL); +insert into tuk values (NULL, 1, NULL); +insert into tik values (NULL, 1, NULL); +insert into tuk values (NULL, 1, 1); +insert into tik values (NULL, 1, 1); +insert into tuk values (NULL, 1, 1); +insert into tik values (NULL, 1, 1); +insert into tuk values (1, NULL, NULL); +insert into tik values (1, NULL, NULL); +insert into tuk values (1, NULL, NULL); +insert into tik values (1, NULL, NULL); +insert into tuk values (1, NULL, 1); +insert into tik values (1, NULL, 1); +insert into tuk values (1, NULL, 1); +insert into tik values (1, NULL, 1); +insert into tuk values (1, 1, NULL); +insert into tik values (1, 1, NULL); +insert into tuk values (1, 1, NULL); +insert into tik values (1, 1, NULL); +insert into tuk values (1, 1, 1); +insert into tik values (1, 1, 1); +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b=1; +select * from tuk where a<=>null and b=1; +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b=1; +select * from tuk where a<=>null and b=1; +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b=1; +select * from tik where a<=>null and b=1; +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b=1; +select * from tik where a<=>null and b=1; +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b>0 and b<2; +select * from tuk where a<=>null and b>0 and b<2; +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b>0 and b<2; +select * from tuk where a<=>null and b>0 and b<2; +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b>0 and b<2; +select * from tik where a<=>null and b>0 and b<2; +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b>0 and b<2; +select * from tik where a<=>null and b>0 and b<2; +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b>=1 and b<2; +select * from tuk where a<=>null and b>=1 and b<2; +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b>=1 and b<2; +select * from tuk where a<=>null and b>=1 and b<2; +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b>=1 and b<2; +select * from tik where a<=>null and b>=1 and b<2; +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b>=1 and b<2; +select * from tik where a<=>null and b>=1 and b<2; +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b=1 and c=1; +select * from tuk where a<=>null and b=1 and c=1; +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b=1 and c=1; +select * from tuk where a<=>null and b=1 and c=1; +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b=1 and c=1; +select * from tik where a<=>null and b=1 and c=1; +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b=1 and c=1; +select * from tik where a<=>null and b=1 and c=1; +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a=1 and b<=>null and c=1; +select * from tuk where a=1 and b<=>null and c=1; +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a=1 and b<=>null and c=1; +select * from tuk where a=1 and b<=>null and c=1; +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a=1 and b<=>null and c=1; +select * from tik where a=1 and b<=>null and c=1; +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a=1 and b<=>null and c=1; +select * from tik where a=1 and b<=>null and c=1; +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b<=>null and c=1; +select * from tuk where a<=>null and b<=>null and c=1; +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b<=>null and c=1; +select * from tuk where a<=>null and b<=>null and c=1; +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b<=>null and c=1; +select * from tik where a<=>null and b<=>null and c=1; +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b<=>null and c=1; +select * from tik where a<=>null and b<=>null and c=1; +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b<=>null and c<=>null; +select * from tuk where a<=>null and b<=>null and c<=>null; +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b<=>null and c<=>null; +select * from tuk where a<=>null and b<=>null and c<=>null; +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b<=>null and c<=>null; +select * from tik where a<=>null and b<=>null and c<=>null; +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b<=>null and c<=>null; +select * from tik where a<=>null and b<=>null and c<=>null; +set @@session.tidb_regard_null_as_point=default; + +# TestIssue30200 +drop table if exists t1; +create table t1(c1 varchar(100), c2 varchar(100), key(c1), key(c2), c3 varchar(100)); +insert into t1 values('ab', '10', '10'); +drop table if exists tt1; +create table tt1(c1 varchar(100), c2 varchar(100), c3 varchar(100), c4 varchar(100), key idx_0(c1), key idx_1(c2, c3)); +insert into tt1 values('ab', '10', '10', '10'); +drop table if exists tt2; +create table tt2 (c1 int , pk int, primary key( pk ) , unique key( c1)); +insert into tt2 values(-3896405, -1), (-2, 1), (-1, -2); +drop table if exists tt3; +create table tt3(c1 int, c2 int, c3 int as (c1 + c2), key(c1), key(c2), key(c3)); +insert into tt3(c1, c2) values(1, 1); +select @@tidb_enable_index_merge; +set tidb_enable_index_merge = on; +explain format=brief select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab'; +select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab'; +explain format=brief select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'ab' or c2 = '10' and char_length(left(c1, 10)) = 10; +select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'ab' or c2 = '10' and char_length(left(c1, 10)) = 10; +explain format=brief select /*+ use_index_merge(tt1) */ 1 from tt1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c3)) = '10'; +select /*+ use_index_merge(tt1) */ 1 from tt1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c3)) = '10'; +explain format=brief select /*+ use_index_merge( tt2 ) */ 1 from tt2 where tt2.c1 in (-3896405) or tt2.pk in (1, 53330) and to_base64(left(pk, 5)); +select /*+ use_index_merge( tt2 ) */ 1 from tt2 where tt2.c1 in (-3896405) or tt2.pk in (1, 53330) and to_base64(left(pk, 5)); +explain format=brief select /*+ use_index_merge(tt3) */ 1 from tt3 where c1 < -10 or c2 < 10 and reverse(c3) = '2'; +select /*+ use_index_merge(tt3) */ 1 from tt3 where c1 < -10 or c2 < 10 and reverse(c3) = '2'; +explain format=brief select 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab'; +select 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab'; +set tidb_enable_index_merge = 1; +set tidb_enable_index_merge = default; + +# TestMultiColMaxOneRow +drop table if exists t1,t2; +create table t1(a int); +create table t2(a int, b int, c int, primary key(a,b) nonclustered); +explain format = 'brief' select (select c from t2 where t2.a = t1.a and t2.b = 1) from t1; +explain format = 'brief' select (select c from t2 where t2.a = t1.a and (t2.b = 1 or t2.b = 2)) from t1; + +# TestSequenceAsDataSource +set tidb_cost_model_version=2; +drop sequence if exists s1, s2; +create sequence s1; +create sequence s2; +explain format = 'brief' select 1 from s1; +explain format = 'brief' select count(1) from s1; +explain format = 'brief' select count(*) from s1; +explain format = 'brief' select sum(1) from s1; +explain format = 'brief' select count(1) as cnt from s1 union select count(1) as cnt from s2; + +# TestOutputSkylinePruningInfo +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int, b int, c int, d int, e int, f int, g int, primary key (a), unique key c_d_e (c, d, e), unique key f (f), unique key f_g (f, g), key g (g)); +set @@tidb_enable_chunk_rpc = on; +--enable_warnings +explain format = 'verbose' select * from t where a > 1 order by f; +explain format = 'verbose' select * from t where f > 1; +explain format = 'verbose' select f from t where f > 1; +explain format = 'verbose' select * from t where f > 3 and g = 5; +explain format = 'verbose' select * from t where g = 5 order by f; +explain format = 'verbose' select * from t where d = 3 order by c, e; +--disable_warnings +set @@tidb_enable_chunk_rpc = default; + +# TestPreferRangeScanForUnsignedIntHandle +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int unsigned primary key, b int, c int, index idx_b(b)); +insert into t values (1,2,3), (4,5,6), (7,8,9), (10,11,12), (13,14,15); +analyze table t; +set @@tidb_enable_chunk_rpc = on; +set tidb_opt_prefer_range_scan = 0; +--enable_warnings +explain format = 'verbose' select * from t where b > 5; +explain format = 'verbose' select * from t where b = 6 order by a limit 1; +explain format = 'verbose' select * from t where b = 6 limit 1; +set tidb_opt_prefer_range_scan = 1; +explain format = 'verbose' select * from t where b > 5; +explain format = 'verbose' select * from t where b = 6 order by a limit 1; +explain format = 'verbose' select * from t where b = 6 limit 1; +--disable_warnings +set @@tidb_enable_chunk_rpc = default; +set tidb_opt_prefer_range_scan = default; + +# TestIssue27083 +drop table if exists t; +create table t(a int primary key, b int, c int, index idx_b(b)); +insert into t values (1,2,3), (4,5,6), (7,8,9), (10, 11, 12), (13,14,15), (16, 17, 18); +analyze table t; +explain format = 'brief' select * from t use index (idx_b) where b = 2 limit 1; + +# TestGroupBySetVar +drop table if exists t1; +create table t1(c1 int); +insert into t1 values(1), (2), (3), (4), (5), (6); +select floor(dt.rn/2) rownum, count(c1) from (select @rownum := @rownum + 1 rn, c1 from (select @rownum := -1) drn, t1) dt group by floor(dt.rn/2) order by rownum; +create table ta(a int, b int); +set sql_mode=''; +explain format = 'brief' select floor(dt.rn/2) rownum, count(c1) from (select @rownum := @rownum + 1 rn, c1 from (select @rownum := -1) drn, t1) dt group by floor(dt.rn/2) order by rownum; +explain format = 'brief' select @n:=@n+1 as e from ta group by e; +explain format = 'brief' select @n:=@n+a as e from ta group by e; +explain format = 'brief' select * from (select @n:=@n+1 as e from ta) tt group by e; +explain format = 'brief' select * from (select @n:=@n+a as e from ta) tt group by e; +explain format = 'brief' select a from ta group by @n:=@n+1; +explain format = 'brief' select a from ta group by @n:=@n+a; +set sql_mode=default; + +# TestDowncastPointGetOrRangeScan +drop table if exists t1, t2, t3, t4, t5, t6, t7, t8; +create table t1 (a bigint key); +create table t2 (a int key); +create definer=`root`@`127.0.0.1` view v1 as (select a from t1) union (select a from t2); +create table t3 (a varchar(100) key); +create table t4 (a varchar(10) key); +create definer=`root`@`127.0.0.1` view v2 as (select a from t3) union (select a from t4); +create table t5 (a char(100) key); +create table t6 (a char(10) key); +create definer=`root`@`127.0.0.1` view v3 as (select a from t5) union (select a from t6); +create table t7 (a varchar(100) key); +create table t8 (a int key); +create definer=`root`@`127.0.0.1` view v4 as (select a from t7) union (select a from t8); +explain format='brief' select * from v1 where a = 1; -- the condition should be downcast through both side and go get point; +select * from v1 where a = 1; -- the condition should be downcast through both side and go get point; +explain format='brief' select * from v1 where a = '1test'; -- the condition should be downcast through both side and go get point too; +select * from v1 where a = '1test'; -- the condition should be downcast through both side and go get point too; +explain format='brief' select * from v1 where a > 1; -- the condition should be downcast through both side and go range scan; +select * from v1 where a > 1; -- the condition should be downcast through both side and go range scan; +explain format='brief' select * from v2 where a = 'test'; +select * from v2 where a = 'test'; +explain format='brief' select * from v2 where a = 1; +select * from v2 where a = 1; +explain format='brief' select * from v2 where a > 'test'; +select * from v2 where a > 'test'; +explain format='brief' select * from v3 where a = 'test' -- the condition shouldn't be downcast through both side and go get point; +select * from v3 where a = 'test' -- the condition shouldn't be downcast through both side and go get point; +explain format='brief' select * from v3 where a > 'test' -- the condition shouldn't be downcast through both side and go get point too; +select * from v3 where a > 'test' -- the condition shouldn't be downcast through both side and go get point too; +explain format='brief' select * from v4 where a = 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range; +select * from v4 where a = 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range; +explain format='brief' select * from v4 where a > 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range; +select * from v4 where a > 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range; + +# TestIssue24095 +set tidb_cost_model_version=2; +drop table if exists t; +create table t (id int, value decimal(10,5)); +desc format = 'brief' select count(*) from t join (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 1) v on v.id = t.id and v.v1 = t.value; +explain format = 'brief' select count(*) from t join (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 1) v on v.id = t.id and v.v1 = t.value; + +# TestFixControl44262 +set tidb_partition_prune_mode='dynamic'; +drop table if exists t1, t2_part; +create table t1 (a int, b int); +create table t2_part (a int, b int, key(a)) partition by hash(a) partitions 4; +--enable_warnings +explain select /*+ TIDB_INLJ(t2_part@sel_2) */ * from t1 where t1.b<10 and not exists (select 1 from t2_part where t1.a=t2_part.a and t2_part.b<20); +--disable_warnings +set @@tidb_opt_fix_control = "44262:ON"; +--enable_warnings +explain select /*+ TIDB_INLJ(t2_part@sel_2) */ * from t1 where t1.b<10 and not exists (select 1 from t2_part where t1.a=t2_part.a and t2_part.b<20); +--disable_warnings + + +# TestPartitionPruningForInExpr +drop table if exists t; +create table t(a int(11) not null, b int) partition by range (a) (partition p0 values less than (4), partition p1 values less than(10), partition p2 values less than maxvalue); +insert into t values (1, 1),(10, 10),(11, 11); +set tidb_opt_fix_control='44262:ON'; +explain format = 'brief' select * from t where a in (1, 2,'11'); +explain format = 'brief' select * from t where a in (17, null); +explain format = 'brief' select * from t where a in (16, 'abc'); +explain format = 'brief' select * from t where a in (15, 0.12, 3.47); +explain format = 'brief' select * from t where a in (0.12, 3.47); +explain format = 'brief' select * from t where a in (14, floor(3.47)); +explain format = 'brief' select * from t where b in (3, 4); +set tidb_opt_fix_control=default; + + +# TestPartitionExplain +drop table if exists pt; +create table pt (id int, c int, key i_id(id), key i_c(c)) partition by range (c) ( +partition p0 values less than (4), +partition p1 values less than (7), +partition p2 values less than (10)); +set @@tidb_enable_index_merge = 1; +set tidb_opt_fix_control='44262:ON'; +--echo ## Table reader +explain format='brief' select * from pt where c > 10; +explain format='brief' select * from pt where c > 8; +explain format='brief' select * from pt where c < 2 or c >= 9; +--echo ## Index reader +explain format='brief' select c from pt; +explain format='brief' select c from pt where c > 10; +explain format='brief' select c from pt where c > 8; +explain format='brief' select c from pt where c < 2 or c >= 9; +--echo ## Index Lookup +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt; +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt where id < 4 and c > 10; +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c > 8; +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c < 2 or c >= 9; +--echo ## Partition selection +explain format='brief' select * from pt partition (p0) where c > 8; +explain format='brief' select c from pt partition (p0, p2) where c > 8; +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt partition (p1, p2) where c < 3 and id = 5; +--echo ## Index Merge +explain format='brief' select * from pt where id = 4 or c < 7; +explain format='brief' select * from pt where id > 4 or c = 7; +set tidb_opt_fix_control=default; +set @@tidb_enable_index_merge = default; + + +# TestIssue41957 +drop table if exists github_events; +CREATE TABLE `github_events` ( + `id` bigint(20) NOT NULL DEFAULT '0', + `type` varchar(29) NOT NULL DEFAULT 'Event', + `created_at` datetime NOT NULL DEFAULT '1970-01-01 00:00:00', + `repo_id` bigint(20) NOT NULL DEFAULT '0', + `repo_name` varchar(140) NOT NULL DEFAULT '', + `actor_id` bigint(20) NOT NULL DEFAULT '0', + `actor_login` varchar(40) NOT NULL DEFAULT '', + `language` varchar(26) NOT NULL DEFAULT '', + `additions` bigint(20) NOT NULL DEFAULT '0', + `deletions` bigint(20) NOT NULL DEFAULT '0', + `action` varchar(11) NOT NULL DEFAULT '', + `number` int(11) NOT NULL DEFAULT '0', + `commit_id` varchar(40) NOT NULL DEFAULT '', + `comment_id` bigint(20) NOT NULL DEFAULT '0', + `org_login` varchar(40) NOT NULL DEFAULT '', + `org_id` bigint(20) NOT NULL DEFAULT '0', + `state` varchar(6) NOT NULL DEFAULT '', + `closed_at` datetime NOT NULL DEFAULT '1970-01-01 00:00:00', + `comments` int(11) NOT NULL DEFAULT '0', + `pr_merged_at` datetime NOT NULL DEFAULT '1970-01-01 00:00:00', + `pr_merged` tinyint(1) NOT NULL DEFAULT '0', + `pr_changed_files` int(11) NOT NULL DEFAULT '0', + `pr_review_comments` int(11) NOT NULL DEFAULT '0', + `pr_or_issue_id` bigint(20) NOT NULL DEFAULT '0', + `event_day` date NOT NULL, + `event_month` date NOT NULL, + `event_year` int(11) NOT NULL, + `push_size` int(11) NOT NULL DEFAULT '0', + `push_distinct_size` int(11) NOT NULL DEFAULT '0', + `creator_user_login` varchar(40) NOT NULL DEFAULT '', + `creator_user_id` bigint(20) NOT NULL DEFAULT '0', + `pr_or_issue_created_at` datetime NOT NULL DEFAULT '1970-01-01 00:00:00', + KEY `index_github_events_on_id` (`id`), + KEY `index_github_events_on_created_at` (`created_at`), + KEY `index_github_events_on_repo_id_type_action_month_actor_login` (`repo_id`,`type`,`action`,`event_month`,`actor_login`), + KEY `index_ge_on_repo_id_type_action_pr_merged_created_at_add_del` (`repo_id`,`type`,`action`,`pr_merged`,`created_at`,`additions`,`deletions`), + KEY `index_ge_on_creator_id_type_action_merged_created_at_add_del` (`creator_user_id`,`type`,`action`,`pr_merged`,`created_at`,`additions`,`deletions`), + KEY `index_ge_on_actor_id_type_action_created_at_repo_id_commits` (`actor_id`,`type`,`action`,`created_at`,`repo_id`,`push_distinct_size`), + KEY `index_ge_on_repo_id_type_action_created_at_number_pdsize_psize` (`repo_id`,`type`,`action`,`created_at`,`number`,`push_distinct_size`,`push_size`), + KEY `index_ge_on_repo_id_type_action_created_at_actor_login` (`repo_id`,`type`,`action`,`created_at`,`actor_login`), + KEY `index_ge_on_repo_name_type` (`repo_name`,`type`), + KEY `index_ge_on_actor_login_type` (`actor_login`,`type`), + KEY `index_ge_on_org_login_type` (`org_login`,`type`), + KEY `index_ge_on_language` (`language`), + KEY `index_ge_on_org_id_type` (`org_id`,`type`), + KEY `index_ge_on_actor_login_lower` ((lower(`actor_login`))), + KEY `index_ge_on_repo_name_lower` ((lower(`repo_name`))), + KEY `index_ge_on_language_lower` ((lower(`language`))), + KEY `index_ge_on_type_action` (`type`,`action`) /*!80000 INVISIBLE */, + KEY `index_ge_on_repo_id_type_created_at` (`repo_id`,`type`,`created_at`), + KEY `index_ge_on_repo_id_created_at` (`repo_id`,`created_at`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY LIST COLUMNS(`type`) +(PARTITION `push_event` VALUES IN ('PushEvent'), + PARTITION `create_event` VALUES IN ('CreateEvent'), + PARTITION `pull_request_event` VALUES IN ('PullRequestEvent'), + PARTITION `watch_event` VALUES IN ('WatchEvent'), + PARTITION `issue_comment_event` VALUES IN ('IssueCommentEvent'), + PARTITION `issues_event` VALUES IN ('IssuesEvent'), + PARTITION `delete_event` VALUES IN ('DeleteEvent'), + PARTITION `fork_event` VALUES IN ('ForkEvent'), + PARTITION `pull_request_review_comment_event` VALUES IN ('PullRequestReviewCommentEvent'), + PARTITION `pull_request_review_event` VALUES IN ('PullRequestReviewEvent'), + PARTITION `gollum_event` VALUES IN ('GollumEvent'), + PARTITION `release_event` VALUES IN ('ReleaseEvent'), + PARTITION `member_event` VALUES IN ('MemberEvent'), + PARTITION `commit_comment_event` VALUES IN ('CommitCommentEvent'), + PARTITION `public_event` VALUES IN ('PublicEvent'), + PARTITION `gist_event` VALUES IN ('GistEvent'), + PARTITION `follow_event` VALUES IN ('FollowEvent'), + PARTITION `event` VALUES IN ('Event'), + PARTITION `download_event` VALUES IN ('DownloadEvent'), + PARTITION `team_add_event` VALUES IN ('TeamAddEvent'), + PARTITION `fork_apply_event` VALUES IN ('ForkApplyEvent')); +SELECT + repo_id, GROUP_CONCAT( + DISTINCT actor_login + ORDER BY cnt DESC + SEPARATOR ',' + ) AS actor_logins +FROM ( + SELECT + ge.repo_id AS repo_id, + ge.actor_login AS actor_login, + COUNT(*) AS cnt + FROM github_events ge + WHERE + type = 'PullRequestEvent' AND action = 'opened' + AND (ge.created_at >= DATE_SUB(NOW(), INTERVAL 1 DAY) AND ge.created_at <= NOW()) + GROUP BY ge.repo_id, ge.actor_login + ORDER BY cnt DESC +) sub +GROUP BY repo_id; + +# TestIssue42588 +drop database if exists testdb; +create database testdb; +use testdb; +drop table if exists `t270`; +CREATE TABLE `t270` ( + `vkey` int(11) DEFAULT NULL, + `pkey` int(11) DEFAULT NULL, + `c1128` varchar(100) DEFAULT NULL, + `c1129` int(11) DEFAULT NULL, + `c1130` varchar(100) DEFAULT NULL, + `c1131` double DEFAULT NULL, + `c1132` varchar(100) DEFAULT NULL, + `c1133` double DEFAULT NULL, + `c1134` varchar(100) DEFAULT NULL, + `c1135` int(11) DEFAULT NULL +); +drop table if exists `t271`; +CREATE TABLE `t271` ( + `vkey` int(11) DEFAULT NULL, + `pkey` int(11) DEFAULT NULL, + `c1136` varchar(100) DEFAULT NULL, + `c1137` int(11) DEFAULT NULL, + `c1138` varchar(100) DEFAULT NULL, + `c1139` int(11) DEFAULT NULL, + `c1140` double DEFAULT NULL, + `c1141` int(11) DEFAULT NULL +); +drop table if exists `t272`; +CREATE TABLE `t272` ( + `vkey` int(11) DEFAULT NULL, + `pkey` int(11) DEFAULT NULL, + `c1142` int(11) DEFAULT NULL, + `c1143` varchar(100) DEFAULT NULL, + `c1144` int(11) DEFAULT NULL, + `c1145` int(11) DEFAULT NULL, + `c1146` varchar(100) DEFAULT NULL, + `c1147` double DEFAULT NULL, + `c1148` varchar(100) DEFAULT NULL, + `c1149` double DEFAULT NULL +); +CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`%` SQL SECURITY DEFINER VIEW `t273_test` (`c0`, `c1`, `c2`, `c3`, `c4`) AS SELECT AVG(37) OVER (PARTITION BY `ref_0`.`c1136` ORDER BY `ref_0`.`vkey` DESC,`ref_0`.`pkey` DESC,`ref_0`.`c1136` DESC,`ref_0`.`c1137`,`ref_0`.`c1138` DESC,`ref_0`.`c1139` DESC,`ref_0`.`c1140` DESC,`ref_0`.`c1141`) AS `c0`,COALESCE(`ref_0`.`c1137`, `ref_0`.`c1141`) AS `c1`,`ref_0`.`vkey` AS `c2`,`ref_0`.`pkey` AS `c3`,`ref_0`.`c1138` AS `c4` FROM `testdb`.`t271` AS `ref_0` WHERE EXISTS (SELECT `subq_0`.`c2` AS `c0`,`subq_0`.`c0` AS `c1`,`subq_0`.`c0` AS `c2`,`subq_0`.`c0` AS `c3`,CASE WHEN EXISTS (SELECT `ref_9`.`c1131` AS `c0`,`ref_9`.`c1131` AS `c1`,(FALSE) XOR (((-45)=(-69)) OR ((-0)>(-71))) AS `c2`,`ref_9`.`c1133` AS `c3`,`ref_9`.`c1128` AS `c4`,-0 AS `c5`,1 AS `c6`,`ref_9`.`c1132` AS `c7`,`ref_9`.`c1131` AS `c8`,`ref_9`.`c1130` AS `c9`,NULL AS `c10` FROM `testdb`.`t270` AS `ref_9` WHERE (-0)<(-8) UNION ALL SELECT `ref_0`.`c1140` AS `c0`,`ref_11`.`c1133` AS `c1`,(NULL)<(NULL) AS `c2`,`ref_0`.`c1140` AS `c3`,`ref_0`.`c1136` AS `c4`,95 AS `c5`,NOT (_UTF8MB4'mum#M' LIKE _UTF8MB4'%_U') AS `c6`,`ref_11`.`c1128` AS `c7`,`ref_11`.`c1131` AS `c8`,(SELECT `c1143` AS `c1143` FROM `testdb`.`t272` ORDER BY `c1143` LIMIT 3,1) AS `c9`,97 AS `c10` FROM `testdb`.`t270` AS `ref_11` WHERE NOT (TRUE)) THEN _UTF8MB4'xf' ELSE _UTF8MB4'>c' END LIKE _UTF8MB4'_^^' AS `c4`,`subq_0`.`c1` AS `c5`,`ref_0`.`vkey` AS `c6`,((`subq_0`.`c1`)=(SELECT `ref_12`.`c1132` AS `c0` FROM `testdb`.`t270` AS `ref_12` WHERE TRUE ORDER BY `c0` DESC LIMIT 1)) XOR ((`ref_0`.`pkey`)>=(SELECT (SELECT `vkey` AS `vkey` FROM `testdb`.`t271` ORDER BY `vkey` LIMIT 1,1) AS `c0` FROM `testdb`.`t271` AS `ref_13` WHERE (-24)<=((SELECT COUNT(`c1140`) AS `count(c1140)` FROM `testdb`.`t271`)) ORDER BY `c0` LIMIT 1)) AS `c7`,`ref_0`.`pkey` AS `c8`,`subq_0`.`c2` AS `c9`,`ref_0`.`vkey` AS `c10`,`ref_0`.`c1139` AS `c11`,TRUE AS `c12`,`subq_0`.`c0` AS `c13`,`subq_0`.`c2` AS `c14`,`subq_0`.`c2` AS `c15`,FALSE AS `c16`,CASE WHEN ((FALSE) OR ((((FALSE) XOR (((-73)<(-91)) OR (((-0) BETWEEN (-0) AND (-0)) AND ((NULL) OR ((0)>((SELECT COUNT(`c1131`) AS `count(c1131)` FROM `testdb`.`t270`))))))) AND ((-19)>(NULL))) OR (((77)<(73)) AND (NOT (((73) IN (SELECT 0 AS `c0` FROM `testdb`.`t271` AS `ref_14` WHERE (NULL) AND (NULL) EXCEPT SELECT NULL AS `c0` FROM `testdb`.`t270` AS `ref_15` WHERE (`ref_15`.`c1131`)!=(SELECT `ref_15`.`c1133` AS `c0` FROM `testdb`.`t270` AS `ref_16` WHERE _UTF8MB4'$@-X' LIKE _UTF8MB4'__%' ORDER BY `c0` DESC LIMIT 1))) IS TRUE))))) OR (NOT ((-24)<=(-43))) THEN `subq_0`.`c1` ELSE `subq_0`.`c2` END AS `c17`,`subq_0`.`c1` AS `c18`,`subq_0`.`c0` AS `c19`,`subq_0`.`c0` AS `c20`,`subq_0`.`c2` AS `c21`,`subq_0`.`c0` AS `c22`,`subq_0`.`c2` AS `c23`,`subq_0`.`c0` AS `c24`,`ref_0`.`c1141` AS `c25` FROM (SELECT DISTINCT TRUE AS `c0`,`ref_1`.`c1143` AS `c1`,`ref_1`.`c1146` AS `c2` FROM `testdb`.`t272` AS `ref_1` WHERE NOT (((`ref_0`.`c1136`)!=(SELECT `ref_2`.`c1146` AS `c0` FROM `testdb`.`t272` AS `ref_2` WHERE (62) BETWEEN ((SELECT COUNT(`c1147`) AS `count(c1147)` FROM `testdb`.`t272`)) AND (-0) ORDER BY `c0` LIMIT 1)) XOR ((-0) BETWEEN (0) AND (-0)))) AS `subq_0` WHERE (CHAR_LENGTH(CASE WHEN ((`subq_0`.`c0`) IS NOT NULL) OR ((`ref_0`.`c1138`)>(SELECT `ref_0`.`c1138` AS `c0` FROM `testdb`.`t272` AS `ref_3` WHERE FALSE ORDER BY `c0` DESC LIMIT 1)) THEN _UTF8MB4'' ELSE _UTF8MB4'tL' END)) BETWEEN (ABS(46%-11)) AND (CASE WHEN (((((`subq_0`.`c2`) IN (SELECT `ref_4`.`c1134` AS `c0` FROM `testdb`.`t270` AS `ref_4` WHERE (NULL LIKE _UTF8MB4'%Ny') OR (EXISTS (SELECT DISTINCT `ref_5`.`c1136` AS `c0`,`ref_5`.`c1140` AS `c1` FROM `testdb`.`t271` AS `ref_5` WHERE FALSE UNION ALL SELECT `ref_4`.`c1130` AS `c0`,`ref_4`.`c1131` AS `c1` FROM `testdb`.`t271` AS `ref_6` WHERE (-97) BETWEEN (73) AND (-10))) UNION ALL SELECT `ref_7`.`c1138` AS `c0` FROM `testdb`.`t271` AS `ref_7` WHERE FALSE)) IS TRUE) OR (NULL)) AND ((NULL)>=((SELECT COUNT(`c1140`) AS `count(c1140)` FROM `testdb`.`t271`)))) XOR (((`ref_0`.`vkey`) IN (SELECT `ref_8`.`c1145` AS `c0` FROM `testdb`.`t272` AS `ref_8` WHERE ((FALSE) AND (NULL)) OR ((`ref_8`.`c1144`) IS NULL))) IS TRUE) THEN 87 ELSE CASE WHEN ((`ref_0`.`c1138`) IS NULL) OR ((-22)!=(-0)) THEN 17 ELSE -67 END END)) ORDER BY `c0` DESC,`c1` DESC,`c2`,`c3`,`c4` DESC; +select + (select + subq_1.c0 as c0 + from + t273_test as ref_84 + where exists ( + select + (select + ref_86.c1147 as c0 + from + t272 as ref_86 + where (subq_1.c0) > (subq_1.c0) + window w0 as (partition by ref_86.c1147 order by ref_86.c1143 desc) + order by c0 limit 1 + ) as c3, + (select + subq_1.c0 as c0 + from + t273_test as ref_89 + order by c0 limit 1) as c4 + from + t271 as ref_85 + ) + order by c0 desc limit 1) as c1 +from + (select 1 as c0) as subq_1; +select + (select + subq_1.c0 as c0 + from + t271 as ref_84 + where exists ( + select + (select + ref_86.c1147 as c0 + from + t272 as ref_86 + where (subq_1.c0) > (subq_1.c0) + window w0 as (partition by ref_86.c1147 order by ref_86.c1143 desc) + order by c0 limit 1 + ) as c3, + (select + subq_1.c0 as c0 + from + t271 as ref_89 + order by c0 limit 1) as c4 + from + t271 as ref_85 + ) + order by c0 desc limit 1) as c1 +from + (select 1 as c0) as subq_1; + +