From 48fcf482faa2f8b96f74aef68186df0d043e73d4 Mon Sep 17 00:00:00 2001 From: Arenatlx <314806019@qq.com> Date: Thu, 31 Aug 2023 12:06:39 +0800 Subject: [PATCH 1/5] This is an automated cherry-pick of #46303 Signed-off-by: ti-chi-bot --- expression/util.go | 155 + planner/core/casetest/integration_test.go | 1608 +++++++ .../testdata/integration_suite_in.json | 613 +++ .../testdata/integration_suite_out.json | 4159 +++++++++++++++++ planner/core/stats.go | 9 +- 5 files changed, 6543 insertions(+), 1 deletion(-) create mode 100644 planner/core/casetest/integration_test.go create mode 100644 planner/core/casetest/testdata/integration_suite_in.json create mode 100644 planner/core/casetest/testdata/integration_suite_out.json diff --git a/expression/util.go b/expression/util.go index 8180158ed40f4..0606aed49815c 100644 --- a/expression/util.go +++ b/expression/util.go @@ -683,6 +683,152 @@ func pushNotAcrossArgs(ctx sessionctx.Context, exprs []Expression, not bool) ([] return newExprs, flag } +// todo: consider more no precision-loss downcast cases. +func noPrecisionLossCastCompatible(cast, argCol *types.FieldType) bool { + // now only consider varchar type and integer. + if !(types.IsTypeVarchar(cast.GetType()) && types.IsTypeVarchar(argCol.GetType())) && + !(mysql.IsIntegerType(cast.GetType()) && mysql.IsIntegerType(argCol.GetType())) { + // varchar type and integer on the storage layer is quite same, while the char type has its padding suffix. + return false + } + if types.IsTypeVarchar(cast.GetType()) { + // cast varchar function only bear the flen extension. + if cast.GetFlen() < argCol.GetFlen() { + return false + } + if !collate.CompatibleCollate(cast.GetCollate(), argCol.GetCollate()) { + return false + } + } else { + // For integers, we should ignore the potential display length represented by flen, using the default flen of the type. + castFlen, _ := mysql.GetDefaultFieldLengthAndDecimal(cast.GetType()) + originFlen, _ := mysql.GetDefaultFieldLengthAndDecimal(argCol.GetType()) + // cast integer function only bear the flen extension and signed symbol unchanged. + if castFlen < originFlen { + return false + } + if mysql.HasUnsignedFlag(cast.GetFlag()) != mysql.HasUnsignedFlag(argCol.GetFlag()) { + return false + } + } + return true +} + +func unwrapCast(sctx sessionctx.Context, parentF *ScalarFunction, castOffset int) (Expression, bool) { + _, collation := parentF.CharsetAndCollation() + cast, ok := parentF.GetArgs()[castOffset].(*ScalarFunction) + if !ok || cast.FuncName.L != ast.Cast { + return parentF, false + } + // eg: if (cast(A) EQ const) with incompatible collation, even if cast is eliminated, the condition still can not be used to build range. + if cast.RetType.EvalType() == types.ETString && !collate.CompatibleCollate(cast.RetType.GetCollate(), collation) { + return parentF, false + } + // 1-castOffset should be constant + if _, ok := parentF.GetArgs()[1-castOffset].(*Constant); !ok { + return parentF, false + } + + // the direct args of cast function should be column. + c, ok := cast.GetArgs()[0].(*Column) + if !ok { + return parentF, false + } + + // current only consider varchar and integer + if !noPrecisionLossCastCompatible(cast.RetType, c.RetType) { + return parentF, false + } + + // the column is covered by indexes, deconstructing it out. + if castOffset == 0 { + return NewFunctionInternal(sctx, parentF.FuncName.L, parentF.RetType, c, parentF.GetArgs()[1]), true + } + return NewFunctionInternal(sctx, parentF.FuncName.L, parentF.RetType, parentF.GetArgs()[0], c), true +} + +// eliminateCastFunction will detect the original arg before and the cast type after, once upon +// there is no precision loss between them, current cast wrapper can be eliminated. For string +// type, collation is also taken into consideration. (mainly used to build range or point) +func eliminateCastFunction(sctx sessionctx.Context, expr Expression) (_ Expression, changed bool) { + f, ok := expr.(*ScalarFunction) + if !ok { + return expr, false + } + _, collation := expr.CharsetAndCollation() + switch f.FuncName.L { + case ast.LogicOr: + dnfItems := FlattenDNFConditions(f) + rmCast := false + rmCastItems := make([]Expression, len(dnfItems)) + for i, dnfItem := range dnfItems { + newExpr, curDowncast := eliminateCastFunction(sctx, dnfItem) + rmCastItems[i] = newExpr + if curDowncast { + rmCast = true + } + } + if rmCast { + // compose the new DNF expression. + return ComposeDNFCondition(sctx, rmCastItems...), true + } + return expr, false + case ast.LogicAnd: + cnfItems := FlattenCNFConditions(f) + rmCast := false + rmCastItems := make([]Expression, len(cnfItems)) + for i, cnfItem := range cnfItems { + newExpr, curDowncast := eliminateCastFunction(sctx, cnfItem) + rmCastItems[i] = newExpr + if curDowncast { + rmCast = true + } + } + if rmCast { + // compose the new CNF expression. + return ComposeCNFCondition(sctx, rmCastItems...), true + } + return expr, false + case ast.EQ, ast.NullEQ, ast.LE, ast.GE, ast.LT, ast.GT: + // for case: eq(cast(test.t2.a, varchar(100), "aaaaa"), once t2.a is covered by index or pk, try deconstructing it out. + if newF, ok := unwrapCast(sctx, f, 0); ok { + return newF, true + } + // for case: eq("aaaaa", cast(test.t2.a, varchar(100)), once t2.a is covered by index or pk, try deconstructing it out. + if newF, ok := unwrapCast(sctx, f, 1); ok { + return newF, true + } + case ast.In: + // case for: cast(a as bigint) in (1,2,3), we could deconstruct column 'a out directly. + cast, ok := f.GetArgs()[0].(*ScalarFunction) + if !ok || cast.FuncName.L != ast.Cast { + return expr, false + } + // eg: if (cast(A) IN {const}) with incompatible collation, even if cast is eliminated, the condition still can not be used to build range. + if cast.RetType.EvalType() == types.ETString && !collate.CompatibleCollate(cast.RetType.GetCollate(), collation) { + return expr, false + } + for _, arg := range f.GetArgs()[1:] { + if _, ok := arg.(*Constant); !ok { + return expr, false + } + } + // the direct args of cast function should be column. + c, ok := cast.GetArgs()[0].(*Column) + if !ok { + return expr, false + } + // current only consider varchar and integer + if !noPrecisionLossCastCompatible(cast.RetType, c.RetType) { + return expr, false + } + newArgs := []Expression{c} + newArgs = append(newArgs, f.GetArgs()[1:]...) + return NewFunctionInternal(sctx, f.FuncName.L, f.RetType, newArgs...), true + } + return expr, false +} + // pushNotAcrossExpr try to eliminate the NOT expr in expression tree. // Input `not` indicates whether there's a `NOT` be pushed down. // Output `changed` indicates whether the output expression differs from the @@ -756,6 +902,15 @@ func PushDownNot(ctx sessionctx.Context, expr Expression) Expression { return newExpr } +// EliminateNoPrecisionLossCast remove the redundant cast function for range build convenience. +// 1: deeper cast embedded in other complicated function will not be considered. +// 2: cast args should be one for original base column and one for constant. +// 3: some collation compatibility and precision loss will be considered when remove this cast func. +func EliminateNoPrecisionLossCast(sctx sessionctx.Context, expr Expression) Expression { + newExpr, _ := eliminateCastFunction(sctx, expr) + return newExpr +} + // ContainOuterNot checks if there is an outer `not`. func ContainOuterNot(expr Expression) bool { return containOuterNot(expr, false) diff --git a/planner/core/casetest/integration_test.go b/planner/core/casetest/integration_test.go new file mode 100644 index 0000000000000..9bdc5ffa7df93 --- /dev/null +++ b/planner/core/casetest/integration_test.go @@ -0,0 +1,1608 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package casetest + +import ( + "fmt" + "strings" + "testing" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/util" + "github.com/stretchr/testify/require" +) + +func TestAggColumnPrune(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into t values(1),(2)") + + var input []string + var output []struct { + SQL string + Res []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) + } +} + +func TestIsFromUnixtimeNullRejective(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a bigint, b bigint);`) + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestSimplifyOuterJoinWithCast(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int not null, b datetime default null)") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestVerboseExplain(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec(`set tidb_opt_limit_push_down_threshold=0`) + tk.MustExec("drop table if exists t1, t2, t3") + tk.MustExec("create table t1(a int, b int)") + tk.MustExec("create table t2(a int, b int)") + tk.MustExec("create table t3(a int, b int, index c(b))") + tk.MustExec("insert into t1 values(1,2)") + tk.MustExec("insert into t1 values(3,4)") + tk.MustExec("insert into t1 values(5,6)") + tk.MustExec("insert into t2 values(1,2)") + tk.MustExec("insert into t2 values(3,4)") + tk.MustExec("insert into t2 values(5,6)") + tk.MustExec("insert into t3 values(1,2)") + tk.MustExec("insert into t3 values(3,4)") + tk.MustExec("insert into t3 values(5,6)") + tk.MustExec("analyze table t1") + tk.MustExec("analyze table t2") + tk.MustExec("analyze table t3") + + // Default RPC encoding may cause statistics explain result differ and then the test unstable. + tk.MustExec("set @@tidb_enable_chunk_rpc = on") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t1" || tblInfo.Name.L == "t2" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestIsolationReadTiFlashNotChoosePointGet(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, primary key (a))") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + var input []string + var output []struct { + SQL string + Result []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) + } +} + +func TestIsolationReadDoNotFilterSystemDB(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("set @@tidb_isolation_read_engines = \"tiflash\"") + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestPartitionTableStats(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + { + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) + tk.MustExec("use test") + tk.MustExec(`set tidb_opt_limit_push_down_threshold=0`) + tk.MustExec("drop table if exists t") + tk.MustExec("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));") + tk.MustExec("insert into t values(21, 1), (22, 2), (23, 3), (24, 4), (15, 5)") + tk.MustExec("analyze table t") + + var input []string + var output []struct { + SQL string + Result []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) + } + } +} + +func TestPartitionPruningForInExpr(t *testing.T) { + failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) + defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("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);") + tk.MustExec("insert into t values (1, 1),(10, 10),(11, 11)") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestMaxMinEliminate(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int primary key)") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("create table cluster_index_t(a int, b int, c int, primary key (a, b));") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestIndexJoinUniqueCompositeIndex(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.MustExec("create table t1(a int not null, c int not null)") + tk.MustExec("create table t2(a int not null, b int not null, c int not null, primary key(a,b))") + tk.MustExec("insert into t1 values(1,1)") + tk.MustExec("insert into t2 values(1,1,1),(1,2,1)") + tk.MustExec("analyze table t1,t2") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestIndexMerge(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, c int, unique index(a), unique index(b), primary key(c))") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +// for issue #14822 and #38258 +func TestIndexJoinTableRange(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int, b int, primary key (a), key idx_t1_b (b))") + tk.MustExec("create table t2(a int, b int, primary key (a), key idx_t1_b (b))") + tk.MustExec("create table t3(a int, b int, c int)") + tk.MustExec("create table t4(a int, b int, c int, primary key (a, b) clustered)") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestSubqueryWithTopN(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int)") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestApproxPercentile(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t values(1, 1), (2, 1), (3, 2), (4, 2), (5, 2)") + + var input []string + var output []struct { + SQL string + Plan []string + Res []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) + } +} + +func TestIndexJoinInnerIndexNDV(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int not null, b int not null, c int not null)") + tk.MustExec("create table t2(a int not null, b int not null, c int not null, index idx1(a,b), index idx2(c))") + tk.MustExec("insert into t1 values(1,1,1),(1,1,1),(1,1,1)") + tk.MustExec("insert into t2 values(1,1,1),(1,1,2),(1,1,3)") + tk.MustExec("analyze table t1, t2") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestIndexMergeSerial(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int, unique key(a), unique key(b))") + tk.MustExec("insert into t value (1, 5), (2, 4), (3, 3), (4, 2), (5, 1)") + tk.MustExec("insert into t value (6, 0), (7, -1), (8, -2), (9, -3), (10, -4)") + tk.MustExec("analyze table t") + + var input []string + var output []struct { + SQL string + Plan []string + Warnings []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) + } +} + +func TestStreamAggProp(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into t values(1),(1),(2)") + + var input []string + var output []struct { + SQL string + Plan []string + Res []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) + } +} + +func TestIndexJoinOnClusteredIndex(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t (a int, b varchar(20), c decimal(40,10), d int, primary key(a,b), key(c))") + tk.MustExec(`insert into t values (1,"111",1.1,11), (2,"222",2.2,12), (3,"333",3.3,13)`) + tk.MustExec("analyze table t") + + var input []string + var output []struct { + SQL string + Plan []string + Res []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery("explain format = 'brief'" + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) + } +} + +func TestPartitionExplain(t *testing.T) { + failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) + defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`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))`) + + tk.MustExec("set @@tidb_enable_index_merge = 1;") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) + }) + tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestIssue20710(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("drop table if exists s;") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("create table s(a int, b int, index(a))") + tk.MustExec("insert into t values(1,1),(1,2),(2,2)") + tk.MustExec("insert into s values(1,1),(2,2),(2,1)") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } +} + +// Apply operator may got panic because empty Projection is eliminated. +func TestIssue23887(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int, b int);") + tk.MustExec("insert into t values(1, 2), (3, 4);") + var input []string + var output []struct { + SQL string + Plan []string + Res []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) + } + + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1 (c1 int primary key, c2 int, c3 int, index c2 (c2));") + tk.MustQuery("select count(1) from (select count(1) from (select * from t1 where c3 = 100) k) k2;").Check(testkit.Rows("1")) +} + +func TestReorderSimplifiedOuterJoins(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t1,t2,t3") + tk.MustExec("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))") + tk.MustExec("create table t2 (pk char(32) primary key nonclustered, col1 varchar(100))") + tk.MustExec("create table t3 (pk char(32) primary key nonclustered, keycol varchar(100), pad1 tinyint(1) default null, pad2 varchar(40), key (keycol,pad1,pad2))") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +// TestIsMatchProp is used to test https://github.com/pingcap/tidb/issues/26017. +func TestIsMatchProp(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int, b int, c int, d int, index idx_a_b_c(a, b, c))") + tk.MustExec("create table t2(a int, b int, c int, d int, index idx_a_b_c_d(a, b, c, d))") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestMergeContinuousSelections(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists ts") + tk.MustExec("create table ts (col_char_64 char(64), col_varchar_64_not_null varchar(64) not null, col_varchar_key varchar(1), id int primary key, col_varchar_64 varchar(64),col_char_64_not_null char(64) not null);") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "ts" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec(" set @@tidb_allow_mpp=1;") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestLimitIndexLookUpKeepOrder(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int, b int, c int, d int, index idx(a,b,c));") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestDecorrelateInnerJoinInSubquery(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int not null, b int not null)") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestDecorrelateLimitInSubquery(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists test") + tk.MustExec("create table test(id int, value int)") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(c int)") + tk.MustExec("insert t values(10), (8), (7), (9), (11)") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestConvertRangeToPoint(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t0") + tk.MustExec("create table t0 (a int, b int, index(a, b))") + tk.MustExec("insert into t0 values (1, 1)") + tk.MustExec("insert into t0 values (2, 2)") + tk.MustExec("insert into t0 values (2, 2)") + tk.MustExec("insert into t0 values (2, 2)") + tk.MustExec("insert into t0 values (2, 2)") + tk.MustExec("insert into t0 values (2, 2)") + tk.MustExec("insert into t0 values (3, 3)") + + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1 (a int, b int, c int, index(a, b, c))") + + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t2 (a float, b float, index(a, b))") + + tk.MustExec("drop table if exists t3") + tk.MustExec("create table t3 (a char(10), b char(10), c char(10), index(a, b, c))") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestIssue22105(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec(`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`) + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestRegardNULLAsPoint(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("drop table if exists tpk") + tk.MustExec(`create table tuk (a int, b int, c int, unique key (a, b, c))`) + tk.MustExec(`create table tik (a int, b int, c int, key (a, b, c))`) + for _, va := range []string{"NULL", "1"} { + for _, vb := range []string{"NULL", "1"} { + for _, vc := range []string{"NULL", "1"} { + tk.MustExec(fmt.Sprintf(`insert into tuk values (%v, %v, %v)`, va, vb, vc)) + tk.MustExec(fmt.Sprintf(`insert into tik values (%v, %v, %v)`, va, vb, vc)) + if va == "1" && vb == "1" && vc == "1" { + continue + } + // duplicated NULL rows + tk.MustExec(fmt.Sprintf(`insert into tuk values (%v, %v, %v)`, va, vb, vc)) + tk.MustExec(fmt.Sprintf(`insert into tik values (%v, %v, %v)`, va, vb, vc)) + } + } + } + + var input []string + var output []struct { + SQL string + PlanEnabled []string + PlanDisabled []string + Result []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + tk.MustExec(`set @@session.tidb_regard_null_as_point=true`) + output[i].PlanEnabled = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + + tk.MustExec(`set @@session.tidb_regard_null_as_point=false`) + output[i].PlanDisabled = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) + }) + tk.MustExec(`set @@session.tidb_regard_null_as_point=true`) + tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].PlanEnabled...)) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) + + tk.MustExec(`set @@session.tidb_regard_null_as_point=false`) + tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].PlanDisabled...)) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) + } +} + +func TestIssue30200(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 varchar(100), c2 varchar(100), key(c1), key(c2), c3 varchar(100));") + tk.MustExec("insert into t1 values('ab', '10', '10');") + + tk.MustExec("drop table if exists tt1;") + tk.MustExec("create table tt1(c1 varchar(100), c2 varchar(100), c3 varchar(100), c4 varchar(100), key idx_0(c1), key idx_1(c2, c3));") + tk.MustExec("insert into tt1 values('ab', '10', '10', '10');") + + tk.MustExec("drop table if exists tt2;") + tk.MustExec("create table tt2 (c1 int , pk int, primary key( pk ) , unique key( c1));") + tk.MustExec("insert into tt2 values(-3896405, -1), (-2, 1), (-1, -2);") + + tk.MustExec("drop table if exists tt3;") + tk.MustExec("create table tt3(c1 int, c2 int, c3 int as (c1 + c2), key(c1), key(c2), key(c3));") + tk.MustExec("insert into tt3(c1, c2) values(1, 1);") + + oriIndexMergeSwitcher := tk.MustQuery("select @@tidb_enable_index_merge;").Rows()[0][0].(string) + tk.MustExec("set tidb_enable_index_merge = on;") + defer func() { + tk.MustExec(fmt.Sprintf("set tidb_enable_index_merge = %s;", oriIndexMergeSwitcher)) + }() + + var input []string + var output []struct { + SQL string + Plan []string + Res []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format=brief " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery("explain format=brief " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) + } +} + +func TestMultiColMaxOneRow(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1,t2") + tk.MustExec("create table t1(a int)") + tk.MustExec("create table t2(a int, b int, c int, primary key(a,b) nonclustered)") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +// TestSequenceAsDataSource is used to test https://github.com/pingcap/tidb/issues/24383. +func TestSequenceAsDataSource(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop sequence if exists s1, s2") + tk.MustExec("create sequence s1") + tk.MustExec("create sequence s2") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestHeuristicIndexSelection(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(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))") + tk.MustExec("create table t2(a int, b int, c int, d int, unique index idx_a (a), unique index idx_b_c (b, c), unique index idx_b_c_a_d (b, c, a, d))") + tk.MustExec("create table t3(a bigint, b varchar(255), c bigint, primary key(a, b) clustered)") + tk.MustExec("create table t4(a bigint, b varchar(255), c bigint, primary key(a, b) nonclustered)") + + // Default RPC encoding may cause statistics explain result differ and then the test unstable. + tk.MustExec("set @@tidb_enable_chunk_rpc = on") + + var input []string + var output []struct { + SQL string + Plan []string + Warnings []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'verbose' " + tt).Rows()) + output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery("explain format = 'verbose' " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) + } +} + +func TestOutputSkylinePruningInfo(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t") + tk.MustExec("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))") + + // Default RPC encoding may cause statistics explain result differ and then the test unstable. + tk.MustExec("set @@tidb_enable_chunk_rpc = on") + + var input []string + var output []struct { + SQL string + Plan []string + Warnings []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'verbose' " + tt).Rows()) + output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery("explain format = 'verbose' " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) + } +} + +func TestPreferRangeScanForUnsignedIntHandle(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int unsigned primary key, b int, c int, index idx_b(b))") + tk.MustExec("insert into t values (1,2,3), (4,5,6), (7,8,9), (10,11,12), (13,14,15)") + do, _ := session.GetDomain(store) + require.Nil(t, do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) + tk.MustExec("analyze table t") + + // Default RPC encoding may cause statistics explain result differ and then the test unstable. + tk.MustExec("set @@tidb_enable_chunk_rpc = on") + + var input []string + var output []struct { + SQL string + Plan []string + Warnings []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) + } +} + +func TestIssue27083(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int primary key, b int, c int, index idx_b(b))") + tk.MustExec("insert into t values (1,2,3), (4,5,6), (7,8,9), (10, 11, 12), (13,14,15), (16, 17, 18)") + do, _ := session.GetDomain(store) + require.Nil(t, do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) + tk.MustExec("analyze table t") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestGroupBySetVar(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1(c1 int);") + tk.MustExec("insert into t1 values(1), (2), (3), (4), (5), (6);") + rows := tk.MustQuery("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;") + rows.Check(testkit.Rows("0 2", "1 2", "2 2")) + + tk.MustExec("create table ta(a int, b int);") + tk.MustExec("set sql_mode='';") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + res := tk.MustQuery("explain format = 'brief' " + tt) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(res.Rows()) + }) + res.Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestIndexMergeWithCorrelatedColumns(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("create table t1(c1 int, c2 int, c3 int, primary key(c1), key(c2));") + tk.MustExec("insert into t1 values(1, 1, 1);") + tk.MustExec("insert into t1 values(2, 2, 2);") + tk.MustExec("create table t2(c1 int, c2 int, c3 int);") + tk.MustExec("insert into t2 values(1, 1, 1);") + tk.MustExec("insert into t2 values(2, 2, 2);") + + tk.MustExec("drop table if exists tt1, tt2;") + tk.MustExec("create table tt1 (c_int int, c_str varchar(40), c_datetime datetime, c_decimal decimal(12, 6), primary key(c_int), key(c_int), key(c_str), unique key(c_decimal), key(c_datetime));") + tk.MustExec("create table tt2 like tt1 ;") + tk.MustExec(`insert into tt1 (c_int, c_str, c_datetime, c_decimal) values (6, 'sharp payne', '2020-06-07 10:40:39', 6.117000) , + (7, 'objective kare', '2020-02-05 18:47:26', 1.053000) , + (8, 'thirsty pasteur', '2020-01-02 13:06:56', 2.506000) , + (9, 'blissful wilbur', '2020-06-04 11:34:04', 9.144000) , + (10, 'reverent mclean', '2020-02-12 07:36:26', 7.751000) ;`) + tk.MustExec(`insert into tt2 (c_int, c_str, c_datetime, c_decimal) values (6, 'beautiful joliot', '2020-01-16 01:44:37', 5.627000) , + (7, 'hopeful blackburn', '2020-05-23 21:44:20', 7.890000) , + (8, 'ecstatic davinci', '2020-02-01 12:27:17', 5.648000) , + (9, 'hopeful lewin', '2020-05-05 05:58:25', 7.288000) , + (10, 'sharp jennings', '2020-01-28 04:35:03', 9.758000) ;`) + + var input []string + var output []struct { + SQL string + Plan []string + Res []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format=brief " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery("explain format=brief " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) + } +} + +func TestIssue31240(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("create table t31240(a int, b int);") + tk.MustExec("set @@tidb_allow_mpp = 0") + tk.MustExec("set tidb_cost_model_version=2") + // since allow-mpp is adjusted to false, there will be no physical plan if TiFlash cop is banned. + tk.MustExec("set @@session.tidb_allow_tiflash_cop=ON") + + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t31240", L: "t31240"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } + tk.MustExec("drop table if exists t31240") +} + +func TestIssue32632(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("CREATE TABLE `partsupp` (" + + " `PS_PARTKEY` bigint(20) NOT NULL," + + "`PS_SUPPKEY` bigint(20) NOT NULL," + + "`PS_AVAILQTY` bigint(20) NOT NULL," + + "`PS_SUPPLYCOST` decimal(15,2) NOT NULL," + + "`PS_COMMENT` varchar(199) NOT NULL," + + "PRIMARY KEY (`PS_PARTKEY`,`PS_SUPPKEY`) /*T![clustered_index] NONCLUSTERED */)") + tk.MustExec("CREATE TABLE `supplier` (" + + "`S_SUPPKEY` bigint(20) NOT NULL," + + "`S_NAME` char(25) NOT NULL," + + "`S_ADDRESS` varchar(40) NOT NULL," + + "`S_NATIONKEY` bigint(20) NOT NULL," + + "`S_PHONE` char(15) NOT NULL," + + "`S_ACCTBAL` decimal(15,2) NOT NULL," + + "`S_COMMENT` varchar(101) NOT NULL," + + "PRIMARY KEY (`S_SUPPKEY`) /*T![clustered_index] CLUSTERED */)") + h := dom.StatsHandle() + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + tk.MustExec("set @@tidb_enforce_mpp = 1") + + tbl1, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "partsupp", L: "partsupp"}) + require.NoError(t, err) + tbl2, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "supplier", L: "supplier"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + tbl2.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + statsTbl1 := h.GetTableStats(tbl1.Meta()) + statsTbl1.RealtimeCount = 800000 + statsTbl2 := h.GetTableStats(tbl2.Meta()) + statsTbl2.RealtimeCount = 10000 + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } + tk.MustExec("drop table if exists partsupp") + tk.MustExec("drop table if exists supplier") +} + +func TestTiFlashPartitionTableScan(t *testing.T) { + failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) + defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") + + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=1") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@tidb_enforce_mpp = on") + tk.MustExec("set @@tidb_allow_batch_cop = 2") + tk.MustExec("drop table if exists rp_t;") + tk.MustExec("drop table if exists hp_t;") + tk.MustExec("create table rp_t(a int) partition by RANGE (a) (PARTITION p0 VALUES LESS THAN (6),PARTITION p1 VALUES LESS THAN (11), PARTITION p2 VALUES LESS THAN (16), PARTITION p3 VALUES LESS THAN (21));") + tk.MustExec("create table hp_t(a int) partition by hash(a) partitions 4;") + tbl1, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "rp_t", L: "rp_t"}) + require.NoError(t, err) + tbl2, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "hp_t", L: "hp_t"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + tbl2.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } + tk.MustExec("drop table rp_t;") + tk.MustExec("drop table hp_t;") +} + +func TestTiFlashFineGrainedShuffle(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@tidb_enforce_mpp = on") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 int, c2 int)") + + tbl1, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t1", L: "t1"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestDowncastPointGetOrRangeScan(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1 (a bigint key)") + tk.MustExec("create table t2 (a int key)") + tk.MustExec("create definer=`root`@`127.0.0.1` view v1 as (select a from t1) union (select a from t2)") + // select * from v where a = 1 will lead a condition: EQ(cast(t2.a as bigint), 1), + // we should downcast it, utilizing t2.a =1 to walking through the pk point-get. Because cast doesn't contain any precision loss. + + tk.MustExec("create table t3 (a varchar(100) key)") + tk.MustExec("create table t4 (a varchar(10) key)") + tk.MustExec("create definer=`root`@`127.0.0.1` view v2 as (select a from t3) union (select a from t4)") + // select * from v2 where a = 'test' will lead a condition: EQ(cast(t2.a as varchar(100) same collation), 1), + // we should downcast it, utilizing t2.a = 'test' to walking through the pk point-get. Because cast doesn't contain any precision loss. + + tk.MustExec("create table t5 (a char(100) key)") + tk.MustExec("create table t6 (a char(10) key)") + tk.MustExec("create definer=`root`@`127.0.0.1` view v3 as (select a from t5) union (select a from t6)") + // select * from v3 where a = 'test' will lead a condition: EQ(cast(t2.a as char(100) same collation), 1), + // for char type, it depends, with binary collate, the appended '0' after cast column a from char(10) to char(100) will make some difference + // on comparison on where a = 'test' before and after the UNION operator; so we didn't allow this kind of type downcast currently (precision diff). + + tk.MustExec("create table t7 (a varchar(100) key)") + tk.MustExec("create table t8 (a int key)") + tk.MustExec("create definer=`root`@`127.0.0.1` view v4 as (select a from t7) union (select a from t8)") + // since UNION OP will unify the a(int) and a(varchar100) as varchar(100) + // select * from v4 where a = "test" will lead a condition: EQ(cast(t2.a as varchar(100)), "test"), and since + // cast int to varchar(100) may have some precision loss, we couldn't utilize a="test" to get the range directly. + + var input []string + var output []struct { + SQL string + Plan []string + Result []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + tt).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + }) + tk.MustQuery("explain format='brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Result...)) + } +} + +func TestNullConditionForPrefixIndex(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`CREATE TABLE t1 ( + id char(1) DEFAULT NULL, + c1 varchar(255) DEFAULT NULL, + c2 text DEFAULT NULL, + KEY idx1 (c1), + KEY idx2 (c1,c2(5)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin`) + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("create table t2(a int, b varchar(10), index idx(b(5)))") + tk.MustExec("create table t3(a int, b varchar(10), c int, primary key (a, b(5)) clustered)") + tk.MustExec("set tidb_opt_prefix_index_single_scan = 1") + tk.MustExec("insert into t1 values ('a', '0xfff', '111111'), ('b', '0xfff', '22 '), ('c', '0xfff', ''), ('d', '0xfff', null)") + tk.MustExec("insert into t2 values (1, 'aaaaaa'), (2, 'bb '), (3, ''), (4, null)") + tk.MustExec("insert into t3 values (1, 'aaaaaa', 2), (1, 'bb ', 3), (1, '', 4)") + + var input []string + var output []struct { + SQL string + Plan []string + Result []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + tt).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + }) + tk.MustQuery("explain format='brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Result...)) + } + + // test plan cache + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk.MustExec("set @@tidb_enable_collect_execution_info=0") + tk.MustExec("prepare stmt from 'select count(1) from t1 where c1 = ? and c2 is not null'") + tk.MustExec("set @a = '0xfff'") + tk.MustQuery("execute stmt using @a").Check(testkit.Rows("3")) + tk.MustQuery("execute stmt using @a").Check(testkit.Rows("3")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustQuery("execute stmt using @a").Check(testkit.Rows("3")) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( + "StreamAgg_17 1.00 root funcs:count(Column#7)->Column#5", + "└─IndexReader_18 1.00 root index:StreamAgg_9", + " └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_16 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo")) +} + +// https://github.com/pingcap/tidb/issues/24095 +func TestIssue24095(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (id int, value decimal(10,5));") + tk.MustExec("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;") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestIndexJoinRangeFallback(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int, b int, c varchar(10), d varchar(10), index idx_a_b_c_d(a, b, c(2), d(2)))") + tk.MustExec("create table t2(e int, f int, g varchar(10), h varchar(10))") + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + setStmt := strings.HasPrefix(tt, "set") + testdata.OnRecord(func() { + output[i].SQL = tt + if !setStmt { + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + } + }) + if setStmt { + tk.MustExec(tt) + } else { + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } + } +} + +func TestFixControl44262(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`set tidb_partition_prune_mode='dynamic'`) + tk.MustExec(`create table t1 (a int, b int)`) + tk.MustExec(`create table t2_part (a int, b int, key(a)) partition by hash(a) partitions 4`) + + testJoin := func(q, join string) { + found := false + for _, x := range tk.MustQuery(`explain ` + q).Rows() { + if strings.Contains(x[0].(string), join) { + found = true + } + } + if !found { + t.Fatal(q, join) + } + } + + testJoin(`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)`, "HashJoin") + tk.MustQuery(`show warnings`).Sort().Check(testkit.Rows( + `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`)) + tk.MustExec(`set @@tidb_opt_fix_control = "44262:ON"`) + testJoin(`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)`, "IndexJoin") + tk.MustQuery(`show warnings`).Sort().Check(testkit.Rows()) // no warning +} diff --git a/planner/core/casetest/testdata/integration_suite_in.json b/planner/core/casetest/testdata/integration_suite_in.json new file mode 100644 index 0000000000000..9a4d4801a4b73 --- /dev/null +++ b/planner/core/casetest/testdata/integration_suite_in.json @@ -0,0 +1,613 @@ +[ + { + "name": "TestPushLimitDownIndexLookUpReader", + "cases": [ + // Limit should be pushed down into IndexLookUpReader, row count of IndexLookUpReader and TableScan should be 1.00. + "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 limit 2,1", + // Projection atop IndexLookUpReader, Limit should be pushed down into IndexLookUpReader, and Projection should have row count 1.00 as well. + "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 order by b desc limit 2,1", + // Limit should be pushed down into IndexLookUpReader when Selection on top of IndexScan. + "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 and c > 1 limit 2,1", + // Limit should NOT be pushed down into IndexLookUpReader when Selection on top of TableScan. + "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 and a > 1 limit 2,1" + ] + }, + { + "name": "TestIsFromUnixtimeNullRejective", + "cases": [ + // fix #12385 + "explain format = 'brief' select * from t t1 left join t t2 on t1.a=t2.a where from_unixtime(t2.b);" + ] + }, + { + "name": "TestAggColumnPrune", + "cases": [ + "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" + ] + }, + { + "name": "TestIndexJoinInnerIndexNDV", + "cases": [ + // t2 should use idx2 instead of idx1, since idx2 has larger NDV. + "explain format = 'brief' select /*+ inl_join(t2) */ * from t1, t2 where t1.a = t2.a and t1.b = t2.b and t1.c = t2.c" + ] + }, + { + "name": "TestSimplifyOuterJoinWithCast", + "cases": [ + // LeftOuterJoin should no be simplified to InnerJoin. + "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'" + ] + }, + { + "name": "TestMaxMinEliminate", + "cases": [ + "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" + ] + }, + { + "name": "TestIndexJoinUniqueCompositeIndex", + "cases": [ + // Row count of IndexScan should be 2. + "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c = t2.c", + // Row count of IndexScan should be 2. + "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c <= t2.b", + // Row count of IndexScan should be 1. + "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t2.b = 1" + ] + }, + { + "name": "TestPartitionTableStats", + "cases": [ + "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" + ] + }, + { + "name": "TestIndexMerge", + "cases": [ + "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 1 or b = 2", + "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, A, B) */ * from t where a = 1 or b = 2", + "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, primary) */ * from t where 1 or t.c", + "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b, c) */ * from t where 1 or t.a = 1 or t.b = 2" + ] + }, + { + "name": "TestSubqueryWithTopN", + "cases": [ + "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" + ] + }, + { + "name": "TestIndexJoinTableRange", + "cases": [ + "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.b", + "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.a and t1.b = t2.b", + "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.a = t4.a where t4.b = 1", + "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.b = t4.b where t4.a = 1" + ] + }, + { + "name": "TestPartitionPruningForInExpr", + "cases": [ + "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)" + ] + }, + { + "name": "TestStreamAggProp", + "cases": [ + "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", + "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" + ] + }, + { + "name": "TestAccessPathOnClusterIndex", + "cases": [ + "select * from t1", + "select * from t1 where t1.a >= 1 and t1.a < 4", + "select * from t1 where t1.a = 1 and t1.b < \"333\"", + "select t1.a, t1.b, t1.c from t1 where t1.c = 3.3", + "select t1.b, t1.c from t1 where t1.c = 2.2", + "select /*+ use_index(t1, c) */ * from t1", + "select * from t1 use index(c) where t1.c in (2.2, 3.3)", + "select * from t1 where t1.a = 1 order by b", + "select * from t1 order by a, b limit 1", + "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a >= 1 or t1.c = 2.2", + "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a = 1 and t1.b = '111' or t1.c = 3.3" + ] + }, + { + "name": "TestIndexJoinOnClusteredIndex", + "cases": [ + "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", + "select /*+ inl_merge_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", + "select /*+ inl_hash_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", + "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a and t1.b = t2.b", + "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.c = t2.c", + "select /*+ inl_merge_join(t1,t2) */ t2.a, t2.c, t2.d from t t1 left join t t2 on t1.a = t2.c;" + ] + }, + { + "name": "TestPartitionExplain", + "cases": [ + // Table reader + "select * from pt where c > 10", + "select * from pt where c > 8", + "select * from pt where c < 2 or c >= 9", + // Index reader + "select c from pt", + "select c from pt where c > 10", + "select c from pt where c > 8", + "select c from pt where c < 2 or c >= 9", + // Index Lookup + "select /*+ use_index(pt, i_id) */ * from pt", + "select /*+ use_index(pt, i_id) */ * from pt where id < 4 and c > 10", + "select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c > 8", + "select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c < 2 or c >= 9", + // Partition selection + "select * from pt partition (p0) where c > 8", + "select c from pt partition (p0, p2) where c > 8", + "select /*+ use_index(pt, i_id) */ * from pt partition (p1, p2) where c < 3 and id = 5", + // Index Merge + "select * from pt where id = 4 or c < 7", + "select * from pt where id > 4 or c = 7" + ] + }, + { + "name": "TestApproxPercentile", + "cases": [ + "select approx_percentile(a, 50) from t", + "select approx_percentile(a, 10) from t", + "select approx_percentile(a, 10+70) from t", + "select approx_percentile(a, 10*10) from t", + "select approx_percentile(a, 50) from t group by b order by b" + ] + }, + { + "name": "TestConvertRangeToPoint", + "cases": [ + "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'" + ] + }, + { + "name": "TestIssue22105", + "cases": [ + "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)" + ] + }, + { + "name": "TestReorderSimplifiedOuterJoins", + "cases": [ + // Query with INNER JOIN or LEFT JOIN should have the same plan. + "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'" + ] + }, + { + "name": "TestDecorrelateInnerJoinInSubquery", + "cases": [ + // Query with WHERE or ON should have the same plan, i.e, the Apply has been decorrelated. + "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)" + ] + }, + { + "name": "TestDecorrelateLimitInSubquery", + "cases": [ + // Query with EXISTS and subquery with LIMIT should have the same plan, i.e, the Limit has been decorrelated. + "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)" + ] + }, + { + "name": "TestMultiColMaxOneRow", + "cases": [ + "select (select c from t2 where t2.a = t1.a and t2.b = 1) from t1", + "select (select c from t2 where t2.a = t1.a and (t2.b = 1 or t2.b = 2)) from t1" + ] + }, + { + "name": "TestIssue24095", + "cases": [ + "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;" + ] + }, + { + "name": "TestSequenceAsDataSource", + "cases": [ + "select 1 from s1", + "select count(1) from s1", + "select count(*) from s1", + "select sum(1) from s1", + "select count(1) as cnt from s1 union select count(1) as cnt from s2" + ] + }, + { + "name": "TestIsMatchProp", + "cases": [ + "select a, b, c from t1 where a > 3 and b = 4 order by a, c", + "select * from t2 where a = 1 and c = 2 order by b, d", + "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", + "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", + "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" + ] + }, + { + "name": "TestHeuristicIndexSelection", + "cases": [ + "select * from t1 where a = 3 or a = 5", + "select f, g from t1 where f = 2 and g in (3, 4, 5)", + "select * from t1 where c = 1 and (d = 2 or d = 3) and e in (4, 5)", + "select f, g from t1 where f = 2 and g > 3", + "select a, b, c from t2 where a = 1 and b = 2 and c in (1, 2, 3, 4, 5)", + "select * from t3 where (a = 1 or a = 3) and b = 'xx'", + "select * from t4 where (a = 1 or a = 3) and b = 'xx'", + "select a, b from t3 where (a = 1 or a = 3) and b = 'xx'", + "select a, b from t4 where (a = 1 or a = 3) and b = 'xx'", + "update t1 set b = 2 where a = 4 or a = 6", + "delete from t1 where f = 2 and g in (3, 4)", + "insert into t3 select a, b, c from t1 where f = 2", + "replace into t3 select a, b, c from t1 where a = 3" + ] + }, + { + "name": "TestOutputSkylinePruningInfo", + "cases": [ + "select * from t where a > 1 order by f", + "select * from t where f > 1", + "select f from t where f > 1", + "select * from t where f > 3 and g = 5", + "select * from t where g = 5 order by f", + "select * from t where d = 3 order by c, e" + ] + }, + { + "name": "TestPreferRangeScanForUnsignedIntHandle", + "cases": [ + "set tidb_opt_prefer_range_scan = 0", + "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" + ] + }, + { + "name": "TestIssue27083", + "cases": [ + "select * from t use index (idx_b) where b = 2 limit 1" + ] + }, + { + "name": "TestGroupBySetVar", + "cases": [ + "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;", + // TODO: fix these two cases + "select @n:=@n+1 as e from ta group by e", + "select @n:=@n+a as e from ta group by e", + "select * from (select @n:=@n+1 as e from ta) tt group by e", + "select * from (select @n:=@n+a as e from ta) tt group by e", + "select a from ta group by @n:=@n+1", + "select a from ta group by @n:=@n+a" + ] + }, + { + "name": "TestIssue30200", + "cases": [ + // to_base64 and from_base64 has not been pushed to TiKV or TiFlash. + // We expect a Selection will be added above IndexMerge. + "select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab';", + // `left` has not been pushed to TiKV, but it has been pushed to TiFlash. + // We expect a Selection will be added above IndexMerge. + "select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'ab' or c2 = '10' and char_length(left(c1, 10)) = 10;", + // c3 is part of idx_1, so it will be put in partial_path's IndexFilters instead of TableFilters. + // But it still cannot be pushed to TiKV. This case cover code in DataSource.buildIndexMergeOrPath. + "select /*+ use_index_merge(tt1) */ 1 from tt1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c3)) = '10';", + // to_base64(left(pk, 5)) is in partial_path's TableFilters. But it cannot be pushed to TiKV. + // So it should be executed in TiDB. This case cover code in DataSource.buildIndexMergeOrPath. + "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));", + // This case covert expression index. + "select /*+ use_index_merge(tt3) */ 1 from tt3 where c1 < -10 or c2 < 10 and reverse(c3) = '2';", + // If no hint, we cannot use index merge if filter cannot be pushed to any storage. + "select 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab';" + ] + }, + { + "name": "TestIndexMergeWithCorrelatedColumns", + "cases": [ + "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", + "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and reverse(c3)) order by c1;", + "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 >= 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", + // Test correlated column in IndexPath.TableFilters. + "select c_int from tt1 where c_decimal < all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt1.c_int = tt2.c_int and tt1.c_datetime > tt2.c_datetime and tt2.c_decimal = 9.060 or tt2.c_str <= 'interesting shtern' and tt1.c_int = tt2.c_int) order by 1;", + // Test correlated column in TablePath.TableFilters. + "select c_int from tt1 where c_decimal > all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt2.c_int = 7 and tt2.c_int < tt1.c_decimal or tt2.c_str >= 'zzzzzzzzzzzzzzzzzzz' and tt1.c_int = tt2.c_int) order by 1;" + ] + }, + { + "name": "TestIssue31240", + "cases": [ + "explain format = 'brief' select count(*) from t31240;", + "set @@tidb_isolation_read_engines=\"tiflash,tidb\";", + "explain format = 'brief' select count(*) from t31240;" + ] + }, + { + "name": "TestVerboseExplain", + "cases": [ + "explain format = 'verbose' select count(*) from t3", + "explain format = 'verbose' select count(*) from t2", + "explain format = 'verbose' select * from t3 order by a", + "explain format = 'verbose' select * from t3 order by b", + "explain format = 'verbose' select * from t3 order by a limit 1", + "explain format = 'verbose' select * from t3 order by b limit 1", + "explain format = 'verbose' select count(*) from t2 group by a", + "explain format = 'verbose' select count(*) from t3 where b = 0", + "explain format = 'verbose' select /*+ use_index(t3, c) */ count(a) from t3 where b = 0", + "explain format = 'verbose' select count(*) from t2 where a = 0", + "explain format = 'verbose' select count(*) from t3 t join t3 on t.a = t3.b", + "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", + "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", + "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", + "explain format = 'verbose' select /*+ merge_join(t1), read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a" + ] + }, + { + "name": "TestRegardNULLAsPoint", + "cases": [ + "select * from tuk where a<=>null and b=1", + "select * from tik where a<=>null and b=1", + "select * from tuk where a<=>null and b>0 and b<2", + "select * from tik where a<=>null and b>0 and b<2", + "select * from tuk where a<=>null and b>=1 and b<2", + "select * from tik where a<=>null and b>=1 and b<2", + "select * from tuk where a<=>null and b=1 and c=1", + "select * from tik where a<=>null and b=1 and c=1", + "select * from tuk where a=1 and b<=>null and c=1", + "select * from tik where a=1 and b<=>null and c=1", + "select * from tuk where a<=>null and b<=>null and c=1", + "select * from tik where a<=>null and b<=>null and c=1", + "select * from tuk where a<=>null and b<=>null and c<=>null", + "select * from tik where a<=>null and b<=>null and c<=>null" + ] + }, + { + "name": "TestIsolationReadDoNotFilterSystemDB", + "cases": [ + "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'", + "desc format = 'brief' select * from information_schema.tables", + "desc format = 'brief' select * from mysql.stats_meta" + ] + }, + { + "name": "TestIsolationReadTiFlashNotChoosePointGet", + "cases": [ + "explain format = 'brief' select * from t where t.a = 1", + "explain format = 'brief' select * from t where t.a in (1, 2)" + ] + }, + { + "name": "TestIssue20710", + "cases": [ + "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" + ] + }, + { + "name": "TestIndexMergeSerial", + "cases": [ + "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", + "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", + "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", + "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)" + ] + }, + { + "name": "TestLimitIndexLookUpKeepOrder", + "cases": [ + "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", + "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10" + ] + }, + { + "name": "TestIssue23887", + "cases": [ + "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t" + ] + }, + { + "name": "TestMergeContinuousSelections", + "cases": [ + "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 ;" + ] + }, + { + "name": "TestPushDownGroupConcatToTiFlash", + "cases": [ + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts", + "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", + "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", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts group by col_2", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts group by col_2", + "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", + "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", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_1, id order by col_0) from ts group by col_2", + "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", + "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", + "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", + "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", + "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", + "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", + "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", + "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", + "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", + "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", + "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", + "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", + "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", + "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", + "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", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'01') from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,1) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,10) from ts group by '010'", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts group by '011'", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts group by 'GG'", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG','GG') from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'Gg','GG') from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG-10','GG') from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct '1200-01-01 00:00:00.023',1200) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0) from ts group by id", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0,id) from ts group by id", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts group by col_1", + "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", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by col_0<=>null) from ts" + ] + }, + { + "name": "TestIssue32632", + "cases": [ + "explain format = 'brief' select sum(ps_supplycost) from partsupp, supplier where ps_suppkey = s_suppkey;" + ] + }, + { + "name": "TestTiFlashPartitionTableScan", + "cases": [ + "explain format = 'brief' select * from rp_t where a = 1 or a = 20", + "explain format = 'brief' select * from hp_t where a = 1 or a = 20", + "explain format = 'brief' select count(*) from rp_t where a = 1 or a = 20", + "explain format = 'brief' select count(*) from hp_t where a = 1 or a = 20" + ] + }, + { + "name": "TestTiFlashFineGrainedShuffle", + "cases": [ + // 1. Can use fine grained shuffle. + "explain format = 'brief' select row_number() over w1 from t1 window w1 as (partition by c1 order by c1);", + // Test two window function. + "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);", + // Limit + Order. + "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;", + // // No partition by key in w2, so disabled. But w1 is still enabled. BUG: https://github.com/pingcap/tidb/pull/35256#discussion_r913324160 + // "explain format = 'brief' select row_number() over w1, row_number() over w2 from t1 window w1 as (partition by c1 order by c1), w2 as (order by c1);", + // GroupBy key and window function partition key are not same. + "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);", + "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);", + // Join, same as GroupBy. + "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);", + // Selection. + "explain format = 'brief' select row_number() over w1 from t1 where c1 < 100 window w1 as (partition by c1 order by c1);", + // 2. Cannot use fine grained shuffle. + // No window function, so disabled. + "explain format = 'brief' select * from t1;", + // No partition key in window function, so disabled. + "explain format = 'brief' select row_number() over w1 from t1 window w1 as (order by c1);", + // GroupBy key is same with window function partition key, so they are in one fragment. + // But fine grained shuffle doesn't support group by for now. + "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);" + ] + }, + { + "name": "TestIndexJoinRangeFallback", + "cases": [ + "set @@tidb_opt_range_max_size = 0", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "set @@tidb_opt_range_max_size = 2900", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "set @@tidb_opt_range_max_size = 2300", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "set @@tidb_opt_range_max_size = 700", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "set @@tidb_opt_range_max_size = 0", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", + "set @@tidb_opt_range_max_size = 300", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", + "set @@tidb_opt_range_max_size = 0", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10", + "set @@tidb_opt_range_max_size = 300", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10" + ] + }, + { + "name": "TestNullConditionForPrefixIndex", + "cases": [ + "select count(1) from t1 where c1 = '0xfff' and c2 is not null", + "select count(1) from t1 where c1 = '0xfff' and c2 is null", + "select count(1) from t1 where c1 >= '0xfff' and c2 is not null", + "select count(1) from t1 where c1 >= '0xfff' and c2 is null", + "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is not null", + "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is null", + "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is not null", + "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is null", + "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is not null", + "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is null", + "select count(1) from t2 use index(idx) where b is not null", + "select count(1) from t2 use index(idx) where b is null", + "select b from t2 use index(idx) where b is not null", + "select b from t2 use index(idx) where b is null", + "select b from t3 where a = 1 and b is not null", + "select b from t3 where a = 1 and b is null" + ] + }, + { + "name": "TestDowncastPointGetOrRangeScan", + "cases": [ + "select * from v1 where a = 1; -- the condition should be downcast through both side and go get point", + "select * from v1 where a = '1test'; -- the condition should be downcast through both side and go get point too", + "select * from v1 where a > 1; -- the condition should be downcast through both side and go range scan", + "select * from v2 where a = 'test';", + "select * from v2 where a = 1;", + "select * from v2 where a > 'test';", + "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 too", + "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" + ] + } +] diff --git a/planner/core/casetest/testdata/integration_suite_out.json b/planner/core/casetest/testdata/integration_suite_out.json new file mode 100644 index 0000000000000..c40e12aa02038 --- /dev/null +++ b/planner/core/casetest/testdata/integration_suite_out.json @@ -0,0 +1,4159 @@ +[ + { + "Name": "TestPushLimitDownIndexLookUpReader", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 limit 2,1", + "Plan": [ + "IndexLookUp 1.00 root limit embedded(offset:2, count:1)", + "├─Limit(Build) 3.00 cop[tikv] offset:0, count:3", + "│ └─IndexRangeScan 3.00 cop[tikv] table:tbl, index:idx_b_c(b, c) range:(1,+inf], keep order:false", + "└─TableRowIDScan(Probe) 1.00 cop[tikv] table:tbl keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 order by b desc limit 2,1", + "Plan": [ + "Projection 1.00 root test.tbl.a, test.tbl.b, test.tbl.c", + "└─IndexLookUp 1.00 root limit embedded(offset:2, count:1)", + " ├─Limit(Build) 3.00 cop[tikv] offset:0, count:3", + " │ └─IndexRangeScan 3.00 cop[tikv] table:tbl, index:idx_b_c(b, c) range:(1,+inf], keep order:true, desc", + " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:tbl keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 and c > 1 limit 2,1", + "Plan": [ + "IndexLookUp 1.00 root limit embedded(offset:2, count:1)", + "├─Limit(Build) 3.00 cop[tikv] offset:0, count:3", + "│ └─Selection 3.00 cop[tikv] gt(test.tbl.c, 1)", + "│ └─IndexRangeScan 3.75 cop[tikv] table:tbl, index:idx_b_c(b, c) range:(1,+inf], keep order:false", + "└─TableRowIDScan(Probe) 1.00 cop[tikv] table:tbl keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 and a > 1 limit 2,1", + "Plan": [ + "Limit 1.00 root offset:2, count:1", + "└─IndexLookUp 3.00 root ", + " ├─IndexRangeScan(Build) 3.75 cop[tikv] table:tbl, index:idx_b_c(b, c) range:(1,+inf], keep order:false", + " └─Limit(Probe) 3.00 cop[tikv] offset:0, count:3", + " └─Selection 3.00 cop[tikv] gt(test.tbl.a, 1)", + " └─TableRowIDScan 3.75 cop[tikv] table:tbl keep order:false" + ] + } + ] + }, + { + "Name": "TestIsFromUnixtimeNullRejective", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t t1 left join t t2 on t1.a=t2.a where from_unixtime(t2.b);", + "Plan": [ + "Projection 9990.00 root test.t.a, test.t.b, test.t.a, test.t.b", + "└─HashJoin 9990.00 root inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─Selection(Build) 7992.00 root from_unixtime(cast(test.t.b, decimal(20,0) BINARY))", + " │ └─TableReader 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.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(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestAggColumnPrune", + "Cases": [ + { + "SQL": "select count(1) from t join (select count(1) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select max(a) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select min(a) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select sum(a) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select avg(a) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select count(1) from t where false group by a) as tmp", + "Res": [ + "0" + ] + }, + { + "SQL": "select count(1) from t join (select max(a) from t where false group by a) as tmp", + "Res": [ + "0" + ] + }, + { + "SQL": "select count(1) from t join (select min(a) from t where false group by a) as tmp", + "Res": [ + "0" + ] + }, + { + "SQL": "select count(1) from t join (select sum(a) from t where false group by a) as tmp", + "Res": [ + "0" + ] + }, + { + "SQL": "select count(1) from t join (select avg(a) from t where false group by a) as tmp", + "Res": [ + "0" + ] + }, + { + "SQL": "SELECT avg(2) FROM(SELECT min(c) FROM t JOIN(SELECT 1 c) d ORDER BY a) e", + "Res": [ + "2.0000" + ] + } + ] + }, + { + "Name": "TestIndexJoinInnerIndexNDV", + "Cases": [ + { + "SQL": "explain format = 'brief' select /*+ inl_join(t2) */ * from t1, t2 where t1.a = t2.a and t1.b = t2.b and t1.c = t2.c", + "Plan": [ + "IndexJoin 3.00 root inner join, inner:IndexLookUp, outer key:test.t1.c, inner key:test.t2.c, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.b, test.t2.b), eq(test.t1.c, test.t2.c)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", + "└─IndexLookUp(Probe) 3.00 root ", + " ├─IndexRangeScan(Build) 3.00 cop[tikv] table:t2, index:idx2(c) range: decided by [eq(test.t2.c, test.t1.c)], keep order:false", + " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t2 keep order:false" + ] + } + ] + }, + { + "Name": "TestSimplifyOuterJoinWithCast", + "Cases": [ + { + "SQL": "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'", + "Plan": [ + "HashJoin 10000.00 root left outer join, equal:[eq(test.t.a, test.t.a)]", + "├─TableReader(Build) 8000.00 root data:Selection", + "│ └─Selection 8000.00 cop[tikv] ge(cast(test.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" + ] + } + ] + }, + { + "Name": "TestMaxMinEliminate", + "Cases": [ + { + "SQL": "explain format = 'brief' (select max(a) from t) union (select min(a) from t)", + "Plan": [ + "HashAgg 2.00 root group by:Column#5, funcs:firstrow(Column#5)->Column#5", + "└─Union 2.00 root ", + " ├─StreamAgg 1.00 root funcs:max(test.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(test.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" + ] + }, + { + "SQL": "explain format = 'brief' select min(a), max(a) from cluster_index_t", + "Plan": [ + "HashJoin 1.00 root CARTESIAN inner join", + "├─StreamAgg(Build) 1.00 root funcs:max(test.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(test.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" + ] + }, + { + "SQL": "explain format = 'brief' select min(b), max(b) from cluster_index_t where a = 1", + "Plan": [ + "HashJoin 1.00 root CARTESIAN inner join", + "├─StreamAgg(Build) 1.00 root funcs:max(test.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(test.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" + ] + }, + { + "SQL": "explain format = 'brief' select min(a), max(a) from cluster_index_t where b = 1", + "Plan": [ + "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(test.cluster_index_t.a)->Column#8, funcs:max(test.cluster_index_t.a)->Column#9", + " └─Selection 10.00 cop[tikv] eq(test.cluster_index_t.b, 1)", + " └─TableFullScan 10000.00 cop[tikv] table:cluster_index_t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select min(b), max(b) from cluster_index_t where b = 1", + "Plan": [ + "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(test.cluster_index_t.b)->Column#8, funcs:max(test.cluster_index_t.b)->Column#9", + " └─Selection 10.00 cop[tikv] eq(test.cluster_index_t.b, 1)", + " └─TableFullScan 10000.00 cop[tikv] table:cluster_index_t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIndexJoinUniqueCompositeIndex", + "Cases": [ + { + "SQL": "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c = t2.c", + "Plan": [ + "IndexJoin 2.00 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.c, test.t2.c)", + "├─TableReader(Build) 1.00 root data:TableFullScan", + "│ └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false", + "└─IndexLookUp(Probe) 2.00 root ", + " ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false", + " └─TableRowIDScan(Probe) 2.00 cop[tikv] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c <= t2.b", + "Plan": [ + "IndexJoin 2.00 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a), other cond:le(test.t1.c, test.t2.b)", + "├─TableReader(Build) 1.00 root data:TableFullScan", + "│ └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false", + "└─IndexLookUp(Probe) 2.00 root ", + " ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(test.t2.a, test.t1.a) le(test.t1.c, test.t2.b)], keep order:false", + " └─TableRowIDScan(Probe) 2.00 cop[tikv] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t2.b = 1", + "Plan": [ + "IndexJoin 1.00 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "├─TableReader(Build) 1.00 root data:TableFullScan", + "│ └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false", + "└─IndexLookUp(Probe) 1.00 root ", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(test.t2.a, test.t1.a) eq(test.t2.b, 1)], keep order:false", + " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2 keep order:false" + ] + } + ] + }, + { + "Name": "TestPartitionTableStats", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t order by a", + "Result": [ + "Sort 10005.00 root test.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" + ] + }, + { + "SQL": "select * from t order by a", + "Result": [ + "15 5", + "21 1", + "22 2", + "23 3", + "24 4" + ] + }, + { + "SQL": "explain format = 'brief' select * from t order by a limit 3", + "Result": [ + "TopN 3.00 root test.t.a, offset:0, count:3", + "└─PartitionUnion 7.00 root ", + " ├─TopN 3.00 root test.t.a, offset:0, count:3", + " │ └─TableReader 3.00 root data:TopN", + " │ └─TopN 3.00 cop[tikv] test.t.a, offset:0, count:3", + " │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + " ├─TopN 1.00 root test.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 test.t.a, offset:0, count:3", + " └─TableReader 3.00 root data:TopN", + " └─TopN 3.00 cop[tikv] test.t.a, offset:0, count:3", + " └─TableFullScan 4.00 cop[tikv] table:t, partition:p2 keep order:false" + ] + }, + { + "SQL": "select * from t order by a limit 3", + "Result": [ + "15 5", + "21 1", + "22 2" + ] + } + ] + }, + { + "Name": "TestIndexMerge", + "Cases": [ + { + "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 1 or b = 2", + "Plan": [ + "IndexMerge 2.00 root type: union", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[2,2], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, A, B) */ * from t where a = 1 or b = 2", + "Plan": [ + "IndexMerge 2.00 root type: union", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[2,2], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, primary) */ * from t where 1 or t.c", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b, c) */ * from t where 1 or t.a = 1 or t.b = 2", + "Plan": [ + "TableReader 10000.00 root data:Selection", + "└─Selection 10000.00 cop[tikv] or(1, or(eq(test.t.a, 1), eq(test.t.b, 2)))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestSubqueryWithTopN", + "Cases": [ + { + "SQL": "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)", + "Plan": [ + "Projection 9990.00 root test.t.b", + "└─Apply 9990.00 root semi join, equal:[eq(test.t.b, test.t.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 7992.00 root not(isnull(test.t.a))", + " └─Projection 9990.00 root test.t.a", + " └─TopN 9990.00 root Column#7, offset:0, count:1", + " └─Projection 9990.00 root test.t.a, plus(test.t.a, test.t.a)->Column#7", + " └─TableReader 9990.00 root data:TopN", + " └─TopN 9990.00 cop[tikv] plus(test.t.a, test.t.a), offset:0, count:1", + " └─TableFullScan 99900000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "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", + "Plan": [ + "Projection 1.00 root test.t.a", + "└─Projection 1.00 root test.t.a, test.t.b, Column#11", + " └─TopN 1.00 root Column#13, offset:0, count:1", + " └─Projection 10000.00 root test.t.a, test.t.b, Column#11, and(eq(test.t.b, 1), Column#11)->Column#13", + " └─HashJoin 10000.00 root left outer semi join, equal:[eq(test.t.b, test.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" + ] + }, + { + "SQL": "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", + "Plan": [ + "TopN 1.00 root Column#4, offset:0, count:1", + "└─Projection 10000.00 root Column#4, test.t.a, test.t.b", + " └─HashJoin 10000.00 root inner join, equal:[eq(test.t.b, Column#4)]", + " ├─Projection(Build) 8000.00 root plus(test.t.b, test.t.b)->Column#4", + " │ └─TableReader 8000.00 root data:Selection", + " │ └─Selection 8000.00 cop[tikv] not(isnull(plus(test.t.b, test.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(test.t.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIndexJoinTableRange", + "Cases": [ + { + "SQL": "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.b", + "Plan": [ + "IndexJoin 12487.50 root inner join, inner:TableReader, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.b, test.t2.b)", + "├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:idx_t1_b(b) keep order:false, stats:pseudo", + "└─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t2.b))", + " └─TableRangeScan 9990.00 cop[tikv] table:t2 range: decided by [test.t1.a], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.a and t1.b = t2.b", + "Plan": [ + "IndexJoin 12487.50 root inner join, inner:TableReader, outer key:test.t1.a, test.t1.b, inner key:test.t2.a, test.t2.a, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.b, test.t2.a), eq(test.t1.b, test.t2.b)", + "├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:idx_t1_b(b) keep order:false, stats:pseudo", + "└─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t2.b))", + " └─TableRangeScan 9990.00 cop[tikv] table:t2 range: decided by [test.t1.a test.t1.b], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.a = t4.a where t4.b = 1", + "Plan": [ + "Projection 12.50 root test.t3.a, test.t3.b, test.t3.c, test.t4.a, test.t4.b, test.t4.c", + "└─IndexJoin 12.50 root inner join, inner:TableReader, outer key:test.t3.a, inner key:test.t4.a, equal cond:eq(test.t3.a, test.t4.a)", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9.99 root data:Selection", + " └─Selection 9.99 cop[tikv] eq(test.t4.b, 1)", + " └─TableRangeScan 9990.00 cop[tikv] table:t4 range: decided by [eq(test.t4.a, test.t3.a) eq(test.t4.b, 1)], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.b = t4.b where t4.a = 1", + "Plan": [ + "Projection 12.50 root test.t3.a, test.t3.b, test.t3.c, test.t4.a, test.t4.b, test.t4.c", + "└─IndexJoin 12.50 root inner join, inner:TableReader, outer key:test.t3.b, inner key:test.t4.b, equal cond:eq(test.t3.b, test.t4.b)", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9.99 root data:Selection", + " └─Selection 9.99 cop[tikv] eq(test.t4.a, 1)", + " └─TableRangeScan 9990.00 cop[tikv] table:t4 range: decided by [eq(test.t4.b, test.t3.b) eq(test.t4.a, 1)], keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestPartitionPruningForInExpr", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t where a in (1, 2,'11')", + "Plan": [ + "TableReader 30.00 root partition:p0,p2 data:Selection", + "└─Selection 30.00 cop[tikv] in(test.t.a, 1, 2, 11)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a in (17, null)", + "Plan": [ + "TableReader 10.00 root partition:p0,p2 data:Selection", + "└─Selection 10.00 cop[tikv] in(test.t.a, 17, NULL)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a in (16, 'abc')", + "Plan": [ + "TableReader 20.00 root partition:p0,p2 data:Selection", + "└─Selection 20.00 cop[tikv] in(test.t.a, 16, 0)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a in (15, 0.12, 3.47)", + "Plan": [ + "TableReader 10.00 root partition:p2 data:Selection", + "└─Selection 10.00 cop[tikv] or(eq(test.t.a, 15), 0)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a in (0.12, 3.47)", + "Plan": [ + "TableDual 0.00 root rows:0" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a in (14, floor(3.47))", + "Plan": [ + "TableReader 20.00 root partition:p0,p2 data:Selection", + "└─Selection 20.00 cop[tikv] in(test.t.a, 14, 3)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b in (3, 4)", + "Plan": [ + "TableReader 20.00 root partition:all data:Selection", + "└─Selection 20.00 cop[tikv] in(test.t.b, 3, 4)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestStreamAggProp", + "Cases": [ + { + "SQL": "select /*+ stream_agg() */ count(*) c from t group by a order by c limit 1", + "Plan": [ + "TopN 1.00 root Column#3, offset:0, count:1", + "└─StreamAgg 8000.00 root group by:test.t.a, funcs:count(1)->Column#3", + " └─Sort 10000.00 root test.t.a", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Res": [ + "1" + ] + }, + { + "SQL": "select /*+ stream_agg() */ count(*) c from t group by a order by c", + "Plan": [ + "Sort 8000.00 root Column#3", + "└─StreamAgg 8000.00 root group by:test.t.a, funcs:count(1)->Column#3", + " └─Sort 10000.00 root test.t.a", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Res": [ + "1", + "2" + ] + }, + { + "SQL": "select /*+ stream_agg() */ count(*) c from t group by a order by a limit 1", + "Plan": [ + "Projection 1.00 root Column#3->Column#4", + "└─Limit 1.00 root offset:0, count:1", + " └─StreamAgg 1.00 root group by:test.t.a, funcs:count(1)->Column#3, funcs:firstrow(test.t.a)->test.t.a", + " └─Sort 1.25 root test.t.a", + " └─TableReader 1.25 root data:TableFullScan", + " └─TableFullScan 1.25 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Res": [ + "2" + ] + }, + { + "SQL": "select /*+ stream_agg() */ count(*) c from t group by a order by a", + "Plan": [ + "Projection 8000.00 root Column#3->Column#4", + "└─StreamAgg 8000.00 root group by:test.t.a, funcs:count(1)->Column#3, funcs:firstrow(test.t.a)->test.t.a", + " └─Sort 10000.00 root test.t.a", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Res": [ + "2", + "1" + ] + } + ] + }, + { + "Name": "TestAccessPathOnClusterIndex", + "Cases": [ + { + "SQL": "select * from t1", + "Plan": [ + "TableReader 3.00 root data:TableFullScan", + "└─TableFullScan 3.00 cop[tikv] table:t1 keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11", + "2 222 2.2000000000 12", + "3 333 3.3000000000 13" + ] + }, + { + "SQL": "select * from t1 where t1.a >= 1 and t1.a < 4", + "Plan": [ + "TableReader 3.00 root data:TableRangeScan", + "└─TableRangeScan 3.00 cop[tikv] table:t1 range:[1,4), keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11", + "2 222 2.2000000000 12", + "3 333 3.3000000000 13" + ] + }, + { + "SQL": "select * from t1 where t1.a = 1 and t1.b < \"333\"", + "Plan": [ + "TableReader 0.82 root data:TableRangeScan", + "└─TableRangeScan 0.82 cop[tikv] table:t1 range:[1 -inf,1 \"333\"), keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11" + ] + }, + { + "SQL": "select t1.a, t1.b, t1.c from t1 where t1.c = 3.3", + "Plan": [ + "IndexReader 1.00 root index:IndexRangeScan", + "└─IndexRangeScan 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false" + ], + "Res": [ + "3 333 3.3000000000" + ] + }, + { + "SQL": "select t1.b, t1.c from t1 where t1.c = 2.2", + "Plan": [ + "IndexReader 1.00 root index:IndexRangeScan", + "└─IndexRangeScan 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false" + ], + "Res": [ + "222 2.2000000000" + ] + }, + { + "SQL": "select /*+ use_index(t1, c) */ * from t1", + "Plan": [ + "IndexLookUp 3.00 root ", + "├─IndexFullScan(Build) 3.00 cop[tikv] table:t1, index:c(c) keep order:false", + "└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t1 keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11", + "2 222 2.2000000000 12", + "3 333 3.3000000000 13" + ] + }, + { + "SQL": "select * from t1 use index(c) where t1.c in (2.2, 3.3)", + "Plan": [ + "IndexLookUp 2.00 root ", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], [3.3000000000,3.3000000000], keep order:false", + "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t1 keep order:false" + ], + "Res": [ + "2 222 2.2000000000 12", + "3 333 3.3000000000 13" + ] + }, + { + "SQL": "select * from t1 where t1.a = 1 order by b", + "Plan": [ + "TableReader 1.00 root data:TableRangeScan", + "└─TableRangeScan 1.00 cop[tikv] table:t1 range:[1,1], keep order:true" + ], + "Res": [ + "1 111 1.1000000000 11" + ] + }, + { + "SQL": "select * from t1 order by a, b limit 1", + "Plan": [ + "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:t1 keep order:true" + ], + "Res": [ + "1 111 1.1000000000 11" + ] + }, + { + "SQL": "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a >= 1 or t1.c = 2.2", + "Plan": [ + "IndexMerge 3.00 root type: union", + "├─TableRangeScan(Build) 3.00 cop[tikv] table:t1 range:[1,+inf], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false", + "└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t1 keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11", + "2 222 2.2000000000 12", + "3 333 3.3000000000 13" + ] + }, + { + "SQL": "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a = 1 and t1.b = '111' or t1.c = 3.3", + "Plan": [ + "IndexMerge 1.67 root type: union", + "├─TableRangeScan(Build) 1.00 cop[tikv] table:t1 range:[1 \"111\",1 \"111\"], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false", + "└─TableRowIDScan(Probe) 1.67 cop[tikv] table:t1 keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11", + "3 333 3.3000000000 13" + ] + } + ] + }, + { + "Name": "TestIndexJoinOnClusteredIndex", + "Cases": [ + { + "SQL": "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", + "Plan": [ + "IndexJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a, equal cond:eq(test.t.a, test.t.a)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", + "└─TableReader(Probe) 3.00 root data:TableRangeScan", + " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a)], keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11 1 111 1.1000000000 11", + "2 222 2.2000000000 12 2 222 2.2000000000 12", + "3 333 3.3000000000 13 3 333 3.3000000000 13" + ] + }, + { + "SQL": "select /*+ inl_merge_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", + "Plan": [ + "IndexMergeJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", + "└─TableReader(Probe) 3.00 root data:TableRangeScan", + " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a)], keep order:true" + ], + "Res": [ + "1 111 1.1000000000 11 1 111 1.1000000000 11", + "2 222 2.2000000000 12 2 222 2.2000000000 12", + "3 333 3.3000000000 13 3 333 3.3000000000 13" + ] + }, + { + "SQL": "select /*+ inl_hash_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", + "Plan": [ + "IndexHashJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a, equal cond:eq(test.t.a, test.t.a)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", + "└─TableReader(Probe) 3.00 root data:TableRangeScan", + " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a)], keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11 1 111 1.1000000000 11", + "2 222 2.2000000000 12 2 222 2.2000000000 12", + "3 333 3.3000000000 13 3 333 3.3000000000 13" + ] + }, + { + "SQL": "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a and t1.b = t2.b", + "Plan": [ + "IndexJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, test.t.b, inner key:test.t.a, test.t.b, equal cond:eq(test.t.a, test.t.a), eq(test.t.b, test.t.b)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", + "└─TableReader(Probe) 3.00 root data:TableRangeScan", + " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a) eq(test.t.b, test.t.b)], keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11 1 111 1.1000000000 11", + "2 222 2.2000000000 12 2 222 2.2000000000 12", + "3 333 3.3000000000 13 3 333 3.3000000000 13" + ] + }, + { + "SQL": "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.c = t2.c", + "Plan": [ + "IndexJoin 3.00 root inner join, inner:IndexLookUp, outer key:test.t.c, inner key:test.t.c, equal cond:eq(test.t.c, test.t.c)", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t.c))", + "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", + "└─IndexLookUp(Probe) 3.00 root ", + " ├─Selection(Build) 3.00 cop[tikv] not(isnull(test.t.c))", + " │ └─IndexRangeScan 3.00 cop[tikv] table:t2, index:c(c) range: decided by [eq(test.t.c, test.t.c)], keep order:false", + " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t2 keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11 1 111 1.1000000000 11", + "2 222 2.2000000000 12 2 222 2.2000000000 12", + "3 333 3.3000000000 13 3 333 3.3000000000 13" + ] + }, + { + "SQL": "select /*+ inl_merge_join(t1,t2) */ t2.a, t2.c, t2.d from t t1 left join t t2 on t1.a = t2.c;", + "Plan": [ + "IndexMergeJoin 3.00 root left outer join, inner:Projection, outer key:Column#9, inner key:test.t.c", + "├─Projection(Build) 3.00 root cast(test.t.a, decimal(10,0) BINARY)->Column#9", + "│ └─IndexReader 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t1, index:c(c) keep order:false", + "└─Projection(Probe) 3.00 root test.t.a, test.t.c, test.t.d", + " └─IndexLookUp 3.00 root ", + " ├─IndexRangeScan(Build) 3.00 cop[tikv] table:t2, index:c(c) range: decided by [eq(test.t.c, Column#9)], keep order:true", + " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t2 keep order:false" + ], + "Res": [ + " ", + " ", + " " + ] + } + ] + }, + { + "Name": "TestPartitionExplain", + "Cases": [ + { + "SQL": "select * from pt where c > 10", + "Plan": [ + "TableReader_7 3333.33 root partition:dual data:Selection_6", + "└─Selection_6 3333.33 cop[tikv] gt(test.pt.c, 10)", + " └─TableFullScan_5 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from pt where c > 8", + "Plan": [ + "TableReader_7 3333.33 root partition:p2 data:Selection_6", + "└─Selection_6 3333.33 cop[tikv] gt(test.pt.c, 8)", + " └─TableFullScan_5 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from pt where c < 2 or c >= 9", + "Plan": [ + "TableReader_7 6656.67 root partition:p0,p2 data:Selection_6", + "└─Selection_6 6656.67 cop[tikv] or(lt(test.pt.c, 2), ge(test.pt.c, 9))", + " └─TableFullScan_5 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select c from pt", + "Plan": [ + "IndexReader_7 10000.00 root partition:all index:IndexFullScan_6", + "└─IndexFullScan_6 10000.00 cop[tikv] table:pt, index:i_c(c) keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select c from pt where c > 10", + "Plan": [ + "IndexReader_6 3333.33 root partition:dual index:IndexRangeScan_5", + "└─IndexRangeScan_5 3333.33 cop[tikv] table:pt, index:i_c(c) range:(10,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select c from pt where c > 8", + "Plan": [ + "IndexReader_6 3333.33 root partition:p2 index:IndexRangeScan_5", + "└─IndexRangeScan_5 3333.33 cop[tikv] table:pt, index:i_c(c) range:(8,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select c from pt where c < 2 or c >= 9", + "Plan": [ + "IndexReader_6 6656.67 root partition:p0,p2 index:IndexRangeScan_5", + "└─IndexRangeScan_5 6656.67 cop[tikv] table:pt, index:i_c(c) range:[-inf,2), [9,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index(pt, i_id) */ * from pt", + "Plan": [ + "IndexLookUp_6 10000.00 root partition:all ", + "├─IndexFullScan_4(Build) 10000.00 cop[tikv] table:pt, index:i_id(id) keep order:false, stats:pseudo", + "└─TableRowIDScan_5(Probe) 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index(pt, i_id) */ * from pt where id < 4 and c > 10", + "Plan": [ + "IndexLookUp_8 1107.78 root partition:dual ", + "├─IndexRangeScan_5(Build) 3323.33 cop[tikv] table:pt, index:i_id(id) range:[-inf,4), keep order:false, stats:pseudo", + "└─Selection_7(Probe) 1107.78 cop[tikv] gt(test.pt.c, 10)", + " └─TableRowIDScan_6 3323.33 cop[tikv] table:pt keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c > 8", + "Plan": [ + "IndexLookUp_8 1107.78 root partition:p2 ", + "├─IndexRangeScan_5(Build) 3323.33 cop[tikv] table:pt, index:i_id(id) range:[-inf,10), keep order:false, stats:pseudo", + "└─Selection_7(Probe) 1107.78 cop[tikv] gt(test.pt.c, 8)", + " └─TableRowIDScan_6 3323.33 cop[tikv] table:pt keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c < 2 or c >= 9", + "Plan": [ + "IndexLookUp_8 5325.33 root partition:p0,p2 ", + "├─IndexFullScan_5(Build) 10000.00 cop[tikv] table:pt, index:i_id(id) keep order:false, stats:pseudo", + "└─Selection_7(Probe) 5325.33 cop[tikv] or(and(lt(test.pt.id, 10), lt(test.pt.c, 2)), ge(test.pt.c, 9))", + " └─TableRowIDScan_6 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from pt partition (p0) where c > 8", + "Plan": [ + "TableReader_7 3333.33 root partition:dual data:Selection_6", + "└─Selection_6 3333.33 cop[tikv] gt(test.pt.c, 8)", + " └─TableFullScan_5 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select c from pt partition (p0, p2) where c > 8", + "Plan": [ + "IndexReader_6 3333.33 root partition:p2 index:IndexRangeScan_5", + "└─IndexRangeScan_5 3333.33 cop[tikv] table:pt, index:i_c(c) range:(8,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index(pt, i_id) */ * from pt partition (p1, p2) where c < 3 and id = 5", + "Plan": [ + "IndexLookUp_8 3.32 root partition:dual ", + "├─IndexRangeScan_5(Build) 10.00 cop[tikv] table:pt, index:i_id(id) range:[5,5], keep order:false, stats:pseudo", + "└─Selection_7(Probe) 3.32 cop[tikv] lt(test.pt.c, 3)", + " └─TableRowIDScan_6 10.00 cop[tikv] table:pt keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from pt where id = 4 or c < 7", + "Plan": [ + "IndexMerge_11 3330.01 root partition:all type: union", + "├─IndexRangeScan_8(Build) 10.00 cop[tikv] table:pt, index:i_id(id) range:[4,4], keep order:false, stats:pseudo", + "├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:pt, index:i_c(c) range:[-inf,7), keep order:false, stats:pseudo", + "└─TableRowIDScan_10(Probe) 3330.01 cop[tikv] table:pt keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from pt where id > 4 or c = 7", + "Plan": [ + "IndexMerge_11 3340.00 root partition:all type: union", + "├─IndexRangeScan_8(Build) 3333.33 cop[tikv] table:pt, index:i_id(id) range:(4,+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:pt, index:i_c(c) range:[7,7], keep order:false, stats:pseudo", + "└─TableRowIDScan_10(Probe) 3340.00 cop[tikv] table:pt keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestApproxPercentile", + "Cases": [ + { + "SQL": "select approx_percentile(a, 50) from t", + "Plan": [ + "HashAgg_5 1.00 root funcs:approx_percentile(test.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" + ], + "Res": [ + "3" + ] + }, + { + "SQL": "select approx_percentile(a, 10) from t", + "Plan": [ + "HashAgg_5 1.00 root funcs:approx_percentile(test.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" + ], + "Res": [ + "1" + ] + }, + { + "SQL": "select approx_percentile(a, 10+70) from t", + "Plan": [ + "HashAgg_5 1.00 root funcs:approx_percentile(test.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" + ], + "Res": [ + "4" + ] + }, + { + "SQL": "select approx_percentile(a, 10*10) from t", + "Plan": [ + "HashAgg_5 1.00 root funcs:approx_percentile(test.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" + ], + "Res": [ + "5" + ] + }, + { + "SQL": "select approx_percentile(a, 50) from t group by b order by b", + "Plan": [ + "Projection_6 8000.00 root Column#4->Column#5", + "└─Sort_7 8000.00 root test.t.b", + " └─HashAgg_9 8000.00 root group by:test.t.b, funcs:approx_percentile(test.t.a, 50)->Column#4, funcs:firstrow(test.t.b)->test.t.b", + " └─TableReader_13 10000.00 root data:TableFullScan_12", + " └─TableFullScan_12 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Res": [ + "1", + "4" + ] + } + ] + }, + { + "Name": "TestConvertRangeToPoint", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t0 where a > 1 and a < 3 order by b limit 2", + "Plan": [ + "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" + ] + }, + { + "SQL": "explain format = 'brief' select * from t1 where a >= 2 and a <= 2 and b = 2 and c > 2", + "Plan": [ + "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" + ] + }, + { + "SQL": "explain format = 'brief' select * from t2 where a >= 2.5 and a <= 2.5 order by b limit 2", + "Plan": [ + "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" + ] + }, + { + "SQL": "explain format = 'brief' select * from t3 where a >= 'a' and a <= 'a' and b = 'b' and c > 'c'", + "Plan": [ + "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" + ] + } + ] + }, + { + "Name": "TestIssue22105", + "Cases": [ + { + "SQL": "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)", + "Plan": [ + "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(test.t1.key4, 42), not(isnull(test.t1.key6))), and(eq(test.t1.key1, 4), eq(test.t1.key3, 6)))", + " └─TableRowIDScan 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestReorderSimplifiedOuterJoins", + "Cases": [ + { + "SQL": "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'", + "Plan": [ + "IndexHashJoin 13.81 root inner join, inner:IndexLookUp, outer key:test.t1.col1, inner key:test.t2.pk, equal cond:eq(test.t1.col1, test.t2.pk)", + "├─IndexHashJoin(Build) 12.50 root inner join, inner:IndexLookUp, outer key:test.t3.pk, inner key:test.t1.col3, equal cond:eq(test.t3.pk, test.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(test.t1.col3))", + "│ │ └─IndexRangeScan 18.12 cop[tikv] table:t1, index:col2(col2, col3) range: decided by [eq(test.t1.col3, test.t3.pk) eq(test.t1.col2, a)], keep order:false, stats:pseudo", + "│ └─Selection(Probe) 12.50 cop[tikv] ne(test.t1.col1, \"aaaaaa\"), ne(test.t1.col1, \"abcdef\"), not(isnull(test.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(test.t2.pk, \"aaaaaa\"), ne(test.t2.pk, \"abcdef\")", + " │ └─IndexRangeScan 12.50 cop[tikv] table:t2, index:PRIMARY(pk) range: decided by [eq(test.t2.pk, test.t1.col1)], keep order:false, stats:pseudo", + " └─Selection(Probe) 12.50 cop[tikv] in(test.t2.col1, \"a\", \"b\")", + " └─TableRowIDScan 12.50 cop[tikv] table:t2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "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'", + "Plan": [ + "IndexHashJoin 13.81 root inner join, inner:IndexLookUp, outer key:test.t1.col1, inner key:test.t2.pk, equal cond:eq(test.t1.col1, test.t2.pk)", + "├─IndexHashJoin(Build) 12.50 root inner join, inner:IndexLookUp, outer key:test.t3.pk, inner key:test.t1.col3, equal cond:eq(test.t3.pk, test.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(test.t1.col3))", + "│ │ └─IndexRangeScan 18.12 cop[tikv] table:t1, index:col2(col2, col3) range: decided by [eq(test.t1.col3, test.t3.pk) eq(test.t1.col2, a)], keep order:false, stats:pseudo", + "│ └─Selection(Probe) 12.50 cop[tikv] ne(test.t1.col1, \"aaaaaa\"), ne(test.t1.col1, \"abcdef\"), not(isnull(test.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(test.t2.pk, \"aaaaaa\"), ne(test.t2.pk, \"abcdef\")", + " │ └─IndexRangeScan 12.50 cop[tikv] table:t2, index:PRIMARY(pk) range: decided by [eq(test.t2.pk, test.t1.col1)], keep order:false, stats:pseudo", + " └─Selection(Probe) 12.50 cop[tikv] in(test.t2.col1, \"a\", \"b\")", + " └─TableRowIDScan 12.50 cop[tikv] table:t2 keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestDecorrelateInnerJoinInSubquery", + "Cases": [ + { + "SQL": "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)", + "Plan": [ + "HashJoin 8000.00 root semi join, equal:[eq(test.t.a, test.t.a)]", + "├─HashJoin(Build) 12500.00 root inner join, equal:[eq(test.t.a, test.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" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 8000.00 root semi join, equal:[eq(test.t.a, test.t.a)]", + "├─HashJoin(Build) 12500.00 root inner join, equal:[eq(test.t.a, test.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" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 10000.00 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─HashAgg(Build) 8000.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", + "│ └─HashJoin 12500.00 root inner join, equal:[eq(test.t.a, test.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" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 10000.00 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─HashAgg(Build) 8000.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", + "│ └─HashJoin 12500.00 root inner join, equal:[eq(test.t.a, test.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" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 10000.00 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─HashAgg(Build) 8000.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", + "│ └─HashJoin 12500.00 root inner join, equal:[eq(test.t.a, test.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" + ] + }, + { + "SQL": "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)", + "Plan": [ + "HashJoin 10000.00 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─HashAgg(Build) 8000.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", + "│ └─HashJoin 12500.00 root inner join, equal:[eq(test.t.a, test.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" + ] + } + ] + }, + { + "Name": "TestDecorrelateLimitInSubquery", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id limit 1)", + "Plan": [ + "HashAgg 1.00 root funcs:count(1)->Column#7", + "└─HashJoin 7992.00 root semi join, equal:[eq(test.test.id, test.test.id)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.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(test.test.id))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id)", + "Plan": [ + "HashAgg 1.00 root funcs:count(1)->Column#7", + "└─HashJoin 7992.00 root semi join, equal:[eq(test.test.id, test.test.id)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.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(test.test.id))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id limit 1,2)", + "Plan": [ + "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(test.test.id, test.test.id)", + " └─TableFullScan 30000000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where 9 in (select c from t s where s.c < t.c limit 3)", + "Plan": [ + "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, test.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(test.t.c, test.t.c)", + " └─TableFullScan 37500.00 cop[tikv] table:s keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestMultiColMaxOneRow", + "Cases": [ + { + "SQL": "select (select c from t2 where t2.a = t1.a and t2.b = 1) from t1", + "Plan": [ + "HashJoin 10000.00 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + "├─TableReader(Build) 10.00 root data:Selection", + "│ └─Selection 10.00 cop[tikv] eq(test.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" + ] + }, + { + "SQL": "select (select c from t2 where t2.a = t1.a and (t2.b = 1 or t2.b = 2)) from t1", + "Plan": [ + "Projection 10000.00 root test.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(test.t2.b, 1), eq(test.t2.b, 2))", + " │ └─IndexRangeScan 100000.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 200.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIssue24095", + "Cases": [ + { + "SQL": "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;", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#10", + "└─HashJoin 1.00 root inner join, equal:[eq(test.t.id, test.t.id) eq(test.t.value, test.t.value)]", + " ├─Selection(Build) 0.80 root not(isnull(test.t.id)), not(isnull(test.t.value))", + " │ └─TopN 1.00 root test.t.value, offset:0, count:1", + " │ └─HashJoin 12487.50 root inner join, equal:[eq(test.t.id, test.t.id)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.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(test.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(test.t.id)), not(isnull(test.t.value))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestSequenceAsDataSource", + "Cases": [ + { + "SQL": "select 1 from s1", + "Plan": [ + "Projection 1.00 root 1->Column#1", + "└─TableDual 1.00 root rows:1" + ] + }, + { + "SQL": "select count(1) from s1", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#1", + "└─TableDual 1.00 root rows:1" + ] + }, + { + "SQL": "select count(*) from s1", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#1", + "└─TableDual 1.00 root rows:1" + ] + }, + { + "SQL": "select sum(1) from s1", + "Plan": [ + "StreamAgg 1.00 root funcs:sum(1)->Column#1", + "└─TableDual 1.00 root rows:1" + ] + }, + { + "SQL": "select count(1) as cnt from s1 union select count(1) as cnt from s2", + "Plan": [ + "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" + ] + } + ] + }, + { + "Name": "TestIsMatchProp", + "Cases": [ + { + "SQL": "select a, b, c from t1 where a > 3 and b = 4 order by a, c", + "Plan": [ + "IndexReader 3.33 root index:Selection", + "└─Selection 3.33 cop[tikv] eq(test.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" + ] + }, + { + "SQL": "select * from t2 where a = 1 and c = 2 order by b, d", + "Plan": [ + "IndexReader 0.01 root index:Selection", + "└─Selection 0.01 cop[tikv] eq(test.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" + ] + }, + { + "SQL": "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", + "Plan": [ + "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" + ] + }, + { + "SQL": "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", + "Plan": [ + "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" + ] + }, + { + "SQL": "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", + "Plan": [ + "IndexReader 0.00 root index:Selection", + "└─Selection 0.00 cop[tikv] eq(test.t2.c, 3), or(and(eq(test.t2.a, 1), and(eq(test.t2.b, 1), lt(test.t2.d, 3))), and(eq(test.t2.a, 1), and(eq(test.t2.b, 1), gt(test.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" + ] + } + ] + }, + { + "Name": "TestHeuristicIndexSelection", + "Cases": [ + { + "SQL": "select * from t1 where a = 3 or a = 5", + "Plan": [ + "Batch_Point_Get_5 2.00 887.04 root table:t1 handle:[3 5], keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 handle of t1 is selected since the path only has point ranges" + ] + }, + { + "SQL": "select f, g from t1 where f = 2 and g in (3, 4, 5)", + "Plan": [ + "Batch_Point_Get_5 3.00 380.16 root table:t1, index:f_g(f, g) keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 unique index f_g of t1 is selected since the path only has point ranges with single scan" + ] + }, + { + "SQL": "select * from t1 where c = 1 and (d = 2 or d = 3) and e in (4, 5)", + "Plan": [ + "Batch_Point_Get_5 4.00 1774.08 root table:t1, index:c_d_e(c, d, e) keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 unique index c_d_e of t1 is selected since the path only has point ranges with double scan" + ] + }, + { + "SQL": "select f, g from t1 where f = 2 and g > 3", + "Plan": [ + "IndexReader_6 33.33 733.82 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 33.33 6783.33 cop[tikv] table:t1, index:f_g(f, g) range:(2 3,2 +inf], keep order:false, stats:pseudo" + ], + "Warnings": [ + "Note 1105 unique index f_g of t1 is selected since the path only fetches limited number of rows with single scan" + ] + }, + { + "SQL": "select a, b, c from t2 where a = 1 and b = 2 and c in (1, 2, 3, 4, 5)", + "Plan": [ + "Selection_6 0.01 289.88 root eq(test.t2.b, 2), in(test.t2.c, 1, 2, 3, 4, 5)", + "└─Point_Get_5 1.00 190.08 root table:t2, index:idx_a(a) " + ], + "Warnings": [ + "Note 1105 unique index idx_a of t2 is selected since the path only has point ranges with double scan" + ] + }, + { + "SQL": "select * from t3 where (a = 1 or a = 3) and b = 'xx'", + "Plan": [ + "Batch_Point_Get_5 2.00 1449.36 root table:t3, clustered index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 handle of t3 is selected since the path only has point ranges" + ] + }, + { + "SQL": "select * from t4 where (a = 1 or a = 3) and b = 'xx'", + "Plan": [ + "Batch_Point_Get_5 2.00 1449.36 root table:t4, index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 unique index PRIMARY of t4 is selected since the path only has point ranges with double scan" + ] + }, + { + "SQL": "select a, b from t3 where (a = 1 or a = 3) and b = 'xx'", + "Plan": [ + "Batch_Point_Get_5 2.00 1322.64 root table:t3, clustered index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 handle of t3 is selected since the path only has point ranges" + ] + }, + { + "SQL": "select a, b from t4 where (a = 1 or a = 3) and b = 'xx'", + "Plan": [ + "Batch_Point_Get_5 2.00 1322.64 root table:t4, index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 unique index PRIMARY of t4 is selected since the path only has point ranges with single scan" + ] + }, + { + "SQL": "update t1 set b = 2 where a = 4 or a = 6", + "Plan": [ + "Update_4 N/A N/A root N/A", + "└─Batch_Point_Get_6 2.00 887.04 root table:t1 handle:[4 6], keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 handle of t1 is selected since the path only has point ranges" + ] + }, + { + "SQL": "delete from t1 where f = 2 and g in (3, 4)", + "Plan": [ + "Delete_4 N/A N/A root N/A", + "└─Selection_7 2.00 493.42 root in(test.t1.g, 3, 4)", + " └─Point_Get_6 1.00 443.52 root table:t1, index:f(f) " + ], + "Warnings": [ + "Note 1105 unique index f of t1 is selected since the path only has point ranges with double scan" + ] + }, + { + "SQL": "insert into t3 select a, b, c from t1 where f = 2", + "Plan": [ + "Insert_1 N/A N/A root N/A", + "└─Projection_6 1.00 253.74 root test.t1.a, test.t1.b, test.t1.c", + " └─Point_Get_7 1.00 253.44 root table:t1, index:f(f) " + ], + "Warnings": [ + "Note 1105 unique index f of t1 is selected since the path only has point ranges with double scan" + ] + }, + { + "SQL": "replace into t3 select a, b, c from t1 where a = 3", + "Plan": [ + "Insert_1 N/A N/A root N/A", + "└─Point_Get_7 1.00 190.08 root table:t1 handle:3" + ], + "Warnings": [ + "Note 1105 handle of t1 is selected since the path only has point ranges" + ] + } + ] + }, + { + "Name": "TestOutputSkylinePruningInfo", + "Cases": [ + { + "SQL": "select * from t where a > 1 order by f", + "Plan": [ + "Sort_5 3333.33 2146348.14 root test.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" + ], + "Warnings": [ + "Note 1105 [t] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}", + "Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [{test.t.f asc}], TaskTp: rootTask}" + ] + }, + { + "SQL": "select * from t where f > 1", + "Plan": [ + "TableReader_7 3333.33 316532.90 root data:Selection_6", + "└─Selection_6 3333.33 3269593.45 cop[tikv] gt(test.t.f, 1)", + " └─TableFullScan_5 10000.00 2770593.45 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warnings": [ + "Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" + ] + }, + { + "SQL": "select f from t where f > 1", + "Plan": [ + "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" + ], + "Warnings": [ + "Note 1105 [f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" + ] + }, + { + "SQL": "select * from t where f > 3 and g = 5", + "Plan": [ + "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(test.t.f, 3)", + " └─TableRowIDScan_13 10.00 2770.59 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warnings": [ + "Note 1105 [t,f_g,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" + ] + }, + { + "SQL": "select * from t where g = 5 order by f", + "Plan": [ + "Sort_5 10.00 21321.97 root test.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" + ], + "Warnings": [ + "Note 1105 [t,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}", + "Note 1105 [t,f_g,g] remain after pruning paths for t given Prop{SortItems: [{test.t.f asc}], TaskTp: rootTask}" + ] + }, + { + "SQL": "select * from t where d = 3 order by c, e", + "Plan": [ + "IndexLookUp_15 10.00 215519.24 root ", + "├─Selection_14(Build) 10.00 2941000.00 cop[tikv] eq(test.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" + ], + "Warnings": [ + "Note 1105 [t] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}", + "Note 1105 [t,c_d_e] remain after pruning paths for t given Prop{SortItems: [{test.t.c asc} {test.t.e asc}], TaskTp: rootTask}" + ] + } + ] + }, + { + "Name": "TestPreferRangeScanForUnsignedIntHandle", + "Cases": [ + { + "SQL": "set tidb_opt_prefer_range_scan = 0", + "Plan": null, + "Warnings": null + }, + { + "SQL": "explain format = 'verbose' select * from t where b > 5", + "Plan": [ + "TableReader_7 3.00 130.42 root data:Selection_6", + "└─Selection_6 3.00 1386.04 cop[tikv] gt(test.t.b, 5)", + " └─TableFullScan_5 5.00 1136.54 cop[tikv] table:t keep order:false" + ], + "Warnings": null + }, + { + "SQL": "explain format = 'verbose' select * from t where b = 6 order by a limit 1", + "Plan": [ + "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(test.t.b, 6)", + " └─TableFullScan_21 5.00 1136.54 cop[tikv] table:t keep order:true" + ], + "Warnings": null + }, + { + "SQL": "explain format = 'verbose' select * from t where b = 6 limit 1", + "Plan": [ + "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(test.t.b, 6)", + " └─TableFullScan_10 5.00 1136.54 cop[tikv] table:t keep order:false" + ], + "Warnings": null + }, + { + "SQL": "set tidb_opt_prefer_range_scan = 1", + "Plan": null, + "Warnings": null + }, + { + "SQL": "explain format = 'verbose' select * from t where b > 5", + "Plan": [ + "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" + ], + "Warnings": [ + "Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" + ] + }, + { + "SQL": "explain format = 'verbose' select * from t where b = 6 order by a limit 1", + "Plan": [ + "TopN_9 0.00 1956.63 root test.t.a, offset:0, count:1", + "└─IndexLookUp_16 0.00 1951.83 root ", + " ├─TopN_15(Build) 0.00 206.70 cop[tikv] test.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" + ], + "Warnings": [ + "Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: copMultiReadTask}" + ] + }, + { + "SQL": "explain format = 'verbose' select * from t where b = 6 limit 1", + "Plan": [ + "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" + ], + "Warnings": [ + "Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: copMultiReadTask}" + ] + } + ] + }, + { + "Name": "TestIssue27083", + "Cases": [ + { + "SQL": "select * from t use index (idx_b) where b = 2 limit 1", + "Plan": [ + "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" + ] + } + ] + }, + { + "Name": "TestGroupBySetVar", + "Cases": [ + { + "SQL": "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;", + "Plan": [ + "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 test.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, test.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" + ] + }, + { + "SQL": "select @n:=@n+1 as e from ta group by e", + "Plan": [ + "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" + ] + }, + { + "SQL": "select @n:=@n+a as e from ta group by e", + "Plan": [ + "Projection 8000.00 root setvar(n, plus(getvar(n), cast(test.ta.a, double BINARY)))->Column#4", + "└─HashAgg 8000.00 root group by:Column#7, funcs:firstrow(Column#6)->test.ta.a", + " └─Projection 10000.00 root test.ta.a->Column#6, setvar(n, plus(getvar(n), cast(test.ta.a, double BINARY)))->Column#7", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from (select @n:=@n+1 as e from ta) tt group by e", + "Plan": [ + "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" + ] + }, + { + "SQL": "select * from (select @n:=@n+a as e from ta) tt group by e", + "Plan": [ + "HashAgg 8000.00 root group by:Column#4, funcs:firstrow(Column#4)->Column#4", + "└─Projection 10000.00 root setvar(n, plus(getvar(n), cast(test.ta.a, double BINARY)))->Column#4", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select a from ta group by @n:=@n+1", + "Plan": [ + "HashAgg 1.00 root group by:Column#5, funcs:firstrow(Column#4)->test.ta.a", + "└─Projection 10000.00 root test.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" + ] + }, + { + "SQL": "select a from ta group by @n:=@n+a", + "Plan": [ + "HashAgg 8000.00 root group by:Column#5, funcs:firstrow(Column#4)->test.ta.a", + "└─Projection 10000.00 root test.ta.a->Column#4, setvar(n, plus(getvar(n), cast(test.ta.a, double BINARY)))->Column#5", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIssue30200", + "Cases": [ + { + "SQL": "select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab';", + "Plan": [ + "Projection 15.99 root 1->Column#5", + "└─Selection 15.99 root or(eq(test.t1.c1, \"de\"), and(eq(test.t1.c2, \"10\"), eq(from_base64(to_base64(test.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" + ], + "Res": [ + "1" + ] + }, + { + "SQL": "select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'ab' or c2 = '10' and char_length(left(c1, 10)) = 10;", + "Plan": [ + "Projection 17.99 root 1->Column#5", + "└─Selection 0.04 root or(eq(test.t1.c1, \"ab\"), and(eq(test.t1.c2, \"10\"), eq(char_length(left(test.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" + ], + "Res": [ + "1" + ] + }, + { + "SQL": "select /*+ use_index_merge(tt1) */ 1 from tt1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c3)) = '10';", + "Plan": [ + "Projection 15.99 root 1->Column#6", + "└─Selection 15.99 root or(eq(test.tt1.c1, \"de\"), and(eq(test.tt1.c2, \"10\"), eq(from_base64(to_base64(test.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" + ], + "Res": [ + "1" + ] + }, + { + "SQL": "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));", + "Plan": [ + "Projection 2.40 root 1->Column#3", + "└─Selection 2.40 root or(eq(test.tt2.c1, -3896405), and(in(test.tt2.pk, 1, 53330), istrue_with_null(cast(to_base64(left(cast(test.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" + ], + "Res": [ + "1" + ] + }, + { + "SQL": "select /*+ use_index_merge(tt3) */ 1 from tt3 where c1 < -10 or c2 < 10 and reverse(c3) = '2';", + "Plan": [ + "Projection 5098.44 root 1->Column#5", + "└─Selection 2825.66 root or(lt(test.tt3.c1, -10), and(lt(test.tt3.c2, 10), eq(reverse(cast(test.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" + ], + "Res": [ + "1" + ] + }, + { + "SQL": "select 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab';", + "Plan": [ + "Projection 8000.00 root 1->Column#5", + "└─Selection 8000.00 root or(eq(test.t1.c1, \"de\"), and(eq(test.t1.c2, \"10\"), eq(from_base64(to_base64(test.t1.c1)), \"ab\")))", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Res": [ + "1" + ] + } + ] + }, + { + "Name": "TestIndexMergeWithCorrelatedColumns", + "Cases": [ + { + "SQL": "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", + "Plan": [ + "Sort 10000.00 root test.t2.c1", + "└─Projection 10000.00 root test.t2.c1, test.t2.c2, test.t2.c3", + " └─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.t2.c1, Column#8), if(ne(Column#9, 0), NULL, 1)), or(eq(Column#10, 0), if(isnull(test.t2.c1), NULL, 0)))", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", + " └─IndexMerge 63.35 root type: union", + " ├─Selection(Build) 10000.00 cop[tikv] eq(10, test.t2.c3)", + " │ └─TableRangeScan 10000.00 cop[tikv] table:t1 range:[10,10], keep order:false, stats:pseudo", + " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", + " │ └─IndexRangeScan 100000.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 63.35 cop[tikv] or(and(eq(test.t1.c1, 10), eq(10, test.t2.c3)), and(eq(test.t1.c2, 1), eq(1, test.t2.c3))), substring(cast(test.t1.c3, var_string(20)), 10)", + " └─TableRowIDScan 89992.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Res": [ + "1 1 1", + "2 2 2" + ] + }, + { + "SQL": "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and reverse(c3)) order by c1;", + "Plan": [ + "Sort 10000.00 root test.t2.c1", + "└─Projection 10000.00 root test.t2.c1, test.t2.c2, test.t2.c3", + " └─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.t2.c1, Column#8), if(ne(Column#9, 0), NULL, 1)), or(eq(Column#10, 0), if(isnull(test.t2.c1), NULL, 0)))", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", + " └─IndexMerge 63.35 root type: union", + " ├─Selection(Build) 10000.00 cop[tikv] eq(10, test.t2.c3)", + " │ └─TableRangeScan 10000.00 cop[tikv] table:t1 range:[10,10], keep order:false, stats:pseudo", + " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", + " │ └─IndexRangeScan 100000.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 63.35 cop[tikv] or(and(eq(test.t1.c1, 10), eq(10, test.t2.c3)), and(eq(test.t1.c2, 1), eq(1, test.t2.c3))), reverse(cast(test.t1.c3, var_string(20)))", + " └─TableRowIDScan 89992.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Res": [ + "2 2 2" + ] + }, + { + "SQL": "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 >= 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", + "Plan": [ + "Sort 10000.00 root test.t2.c1", + "└─Projection 10000.00 root test.t2.c1, test.t2.c2, test.t2.c3", + " └─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.t2.c1, Column#8), if(ne(Column#9, 0), NULL, 1)), or(eq(Column#10, 0), if(isnull(test.t2.c1), NULL, 0)))", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", + " └─IndexMerge 30263.46 root type: union", + " ├─Selection(Build) 33333.33 cop[tikv] eq(test.t1.c1, test.t2.c3)", + " │ └─TableRangeScan 33333333.33 cop[tikv] table:t1 range:[10,+inf], keep order:false, stats:pseudo", + " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", + " │ └─IndexRangeScan 100000.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 30263.46 cop[tikv] or(and(ge(test.t1.c1, 10), eq(test.t1.c1, test.t2.c3)), and(eq(test.t1.c2, 1), eq(1, test.t2.c3))), substring(cast(test.t1.c3, var_string(20)), 10)", + " └─TableRowIDScan 33386666.67 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Res": [ + "1 1 1", + "2 2 2" + ] + }, + { + "SQL": "select c_int from tt1 where c_decimal < all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt1.c_int = tt2.c_int and tt1.c_datetime > tt2.c_datetime and tt2.c_decimal = 9.060 or tt2.c_str <= 'interesting shtern' and tt1.c_int = tt2.c_int) order by 1;", + "Plan": [ + "Projection 10000.00 root test.tt1.c_int", + "└─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.tt1.c_decimal, Column#9), if(ne(Column#10, 0), NULL, 1)), or(eq(Column#11, 0), if(isnull(test.tt1.c_decimal), NULL, 0)))", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:tt1 keep order:true, stats:pseudo", + " └─StreamAgg(Probe) 10000.00 root funcs:min(Column#14)->Column#9, funcs:sum(Column#15)->Column#10, funcs:count(1)->Column#11", + " └─Projection 11.05 root test.tt2.c_decimal->Column#14, cast(isnull(test.tt2.c_decimal), decimal(20,0) BINARY)->Column#15", + " └─IndexMerge 11.05 root type: union", + " ├─Selection(Build) 10.00 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", + " │ └─IndexRangeScan 10000.00 cop[tikv] table:tt2, index:c_decimal(c_decimal) range:[9.060000,9.060000], keep order:false, stats:pseudo", + " ├─Selection(Build) 33233.33 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", + " │ └─IndexRangeScan 33233333.33 cop[tikv] table:tt2, index:c_str(c_str) range:[-inf,\"interesting shtern\"], keep order:false, stats:pseudo", + " └─Selection(Probe) 11.05 cop[tikv] or(and(eq(test.tt1.c_int, test.tt2.c_int), and(gt(test.tt1.c_datetime, test.tt2.c_datetime), eq(test.tt2.c_decimal, 9.060))), and(le(test.tt2.c_str, \"interesting shtern\"), eq(test.tt1.c_int, test.tt2.c_int)))", + " └─TableRowIDScan 33243.33 cop[tikv] table:tt2 keep order:false, stats:pseudo" + ], + "Res": [ + "7", + "8", + "10" + ] + }, + { + "SQL": "select c_int from tt1 where c_decimal > all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt2.c_int = 7 and tt2.c_int < tt1.c_decimal or tt2.c_str >= 'zzzzzzzzzzzzzzzzzzz' and tt1.c_int = tt2.c_int) order by 1;", + "Plan": [ + "Projection 10000.00 root test.tt1.c_int", + "└─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(gt(test.tt1.c_decimal, Column#9), if(ne(Column#10, 0), NULL, 1)), or(eq(Column#11, 0), if(isnull(test.tt1.c_decimal), NULL, 0)))", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:tt1 keep order:true, stats:pseudo", + " └─StreamAgg(Probe) 10000.00 root funcs:max(Column#14)->Column#9, funcs:sum(Column#15)->Column#10, funcs:count(1)->Column#11", + " └─Projection 17.91 root test.tt2.c_decimal->Column#14, cast(isnull(test.tt2.c_decimal), decimal(20,0) BINARY)->Column#15", + " └─IndexMerge 17.91 root type: union", + " ├─Selection(Build) 10000.00 cop[tikv] lt(7, test.tt1.c_decimal)", + " │ └─TableRangeScan 10000.00 cop[tikv] table:tt2 range:[7,7], keep order:false, stats:pseudo", + " ├─Selection(Build) 33333.33 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", + " │ └─IndexRangeScan 33333333.33 cop[tikv] table:tt2, index:c_str(c_str) range:[\"zzzzzzzzzzzzzzzzzzz\",+inf], keep order:false, stats:pseudo", + " └─Selection(Probe) 17.91 cop[tikv] or(and(eq(test.tt2.c_int, 7), lt(7, test.tt1.c_decimal)), and(ge(test.tt2.c_str, \"zzzzzzzzzzzzzzzzzzz\"), eq(test.tt1.c_int, test.tt2.c_int)))", + " └─TableRowIDScan 43330.00 cop[tikv] table:tt2 keep order:false, stats:pseudo" + ], + "Res": [ + "6", + "7", + "8", + "9" + ] + } + ] + }, + { + "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_19 3.00 162366.01 root CARTESIAN left outer semi join", + "├─Selection_36(Build) 0.80 31149.25 root eq(2, Column#18)", + "│ └─StreamAgg_43 1.00 31099.35 root funcs:count(1)->Column#18", + "│ └─TableReader_55 3.00 30949.65 root MppVersion: 2, data:ExchangeSender_54", + "│ └─ExchangeSender_54 3.00 464139.20 mpp[tiflash] ExchangeType: PassThrough", + "│ └─TableFullScan_53 3.00 464139.20 mpp[tiflash] table:t1 keep order:false", + "└─Projection_20(Probe) 3.00 129648.62 root 1->Column#28", + " └─Apply_22 3.00 129648.32 root CARTESIAN left outer join", + " ├─IndexReader_26(Build) 3.00 53.37 root index:IndexFullScan_25", + " │ └─IndexFullScan_25 3.00 610.50 cop[tikv] table:t, index:c(b) keep order:false", + " └─Projection_27(Probe) 3.00 43198.32 root 1->Column#26", + " └─Limit_30 3.00 43198.22 root offset:0, count:1", + " └─TableReader_35 3.00 43198.22 root MppVersion: 2, data:ExchangeSender_34", + " └─ExchangeSender_34 3.00 647920.44 mpp[tiflash] ExchangeType: PassThrough", + " └─Limit_33 3.00 647920.44 mpp[tiflash] offset:0, count:1", + " └─TableFullScan_32 3.00 647920.44 mpp[tiflash] table:t2 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": "TestRegardNULLAsPoint", + "Cases": [ + { + "SQL": "select * from tuk where a<=>null and b=1", + "PlanEnabled": [ + "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" + ], + "PlanDisabled": [ + "IndexReader_7 0.01 root index:Selection_6", + "└─Selection_6 0.01 cop[tikv] eq(test.tuk.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b=1", + "PlanEnabled": [ + "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" + ], + "PlanDisabled": [ + "IndexReader_7 0.01 root index:Selection_6", + "└─Selection_6 0.01 cop[tikv] eq(test.tik.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b>0 and b<2", + "PlanEnabled": [ + "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" + ], + "PlanDisabled": [ + "IndexReader_7 0.25 root index:Selection_6", + "└─Selection_6 0.25 cop[tikv] eq(test.tuk.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b>0 and b<2", + "PlanEnabled": [ + "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" + ], + "PlanDisabled": [ + "IndexReader_7 0.25 root index:Selection_6", + "└─Selection_6 0.25 cop[tikv] eq(test.tik.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b>=1 and b<2", + "PlanEnabled": [ + "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" + ], + "PlanDisabled": [ + "IndexReader_7 0.25 root index:Selection_6", + "└─Selection_6 0.25 cop[tikv] eq(test.tuk.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b>=1 and b<2", + "PlanEnabled": [ + "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" + ], + "PlanDisabled": [ + "IndexReader_7 0.25 root index:Selection_6", + "└─Selection_6 0.25 cop[tikv] eq(test.tik.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b=1 and c=1", + "PlanEnabled": [ + "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" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tuk.b, 1), eq(test.tuk.c, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b=1 and c=1", + "PlanEnabled": [ + "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" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tik.b, 1), eq(test.tik.c, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tuk where a=1 and b<=>null and c=1", + "PlanEnabled": [ + "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" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.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" + ], + "Result": [ + "1 1", + "1 1" + ] + }, + { + "SQL": "select * from tik where a=1 and b<=>null and c=1", + "PlanEnabled": [ + "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" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.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" + ], + "Result": [ + "1 1", + "1 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b<=>null and c=1", + "PlanEnabled": [ + "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" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tuk.c, 1), nulleq(test.tuk.b, NULL)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1", + " 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b<=>null and c=1", + "PlanEnabled": [ + "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" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tik.c, 1), nulleq(test.tik.b, NULL)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1", + " 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b<=>null and c<=>null", + "PlanEnabled": [ + "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" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] nulleq(test.tuk.b, NULL), nulleq(test.tuk.c, NULL)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " ", + " " + ] + }, + { + "SQL": "select * from tik where a<=>null and b<=>null and c<=>null", + "PlanEnabled": [ + "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" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] nulleq(test.tik.b, NULL), nulleq(test.tik.c, NULL)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " ", + " " + ] + } + ] + }, + { + "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": "TestIssue20710", + "Cases": [ + { + "SQL": "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.b", + "Plan": [ + "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.b)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.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(test.s.a))", + " │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", + " └─Selection(Probe) 12475.01 cop[tikv] not(isnull(test.s.b))", + " └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.a", + "Plan": [ + "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.a)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.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(test.s.a))", + " │ └─IndexRangeScan 12487.50 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:s keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.a = s.b", + "Plan": [ + "Projection 12475.01 root test.t.a, test.t.b, test.s.a, test.s.b", + "└─IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.a, test.s.b)", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.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(test.s.a))", + " │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", + " └─Selection(Probe) 12475.01 cop[tikv] not(isnull(test.s.b))", + " └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.b", + "Plan": [ + "IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.b)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.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(test.s.a))", + " │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", + " └─Selection(Probe) 12475.01 cop[tikv] not(isnull(test.s.b))", + " └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.a", + "Plan": [ + "IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.a)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.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(test.s.a))", + " │ └─IndexRangeScan 12487.50 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:s keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.a = s.b", + "Plan": [ + "Projection 12475.01 root test.t.a, test.t.b, test.s.a, test.s.b", + "└─IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.a, test.s.b)", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.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(test.s.a))", + " │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", + " └─Selection(Probe) 12475.01 cop[tikv] not(isnull(test.s.b))", + " └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIndexMergeSerial", + "Cases": [ + { + "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", + "Plan": [ + "IndexMerge 8.00 root type: union", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─Selection(Build) 1.00 cop[tikv] 1", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" + ], + "Warnings": null + }, + { + "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", + "Plan": [ + "IndexMerge 8.00 root type: union", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─Selection(Build) 1.00 cop[tikv] 1", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" + ], + "Warnings": null + }, + { + "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", + "Plan": [ + "IndexMerge 8.00 root type: union", + "├─Selection(Build) 1.00 cop[tikv] 1", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─Selection(Build) 1.00 cop[tikv] 1", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" + ], + "Warnings": null + }, + { + "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)", + "Plan": [ + "IndexMerge 0.29 root type: union", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─Selection(Probe) 0.29 cop[tikv] or(and(eq(test.t.a, 1), eq(length(cast(test.t.b, var_string(20))), 1)), and(eq(test.t.b, 1), eq(length(cast(test.t.a, var_string(20))), 1)))", + " └─TableRowIDScan 1.90 cop[tikv] table:t keep order:false" + ], + "Warnings": null + } + ] + }, + { + "Name": "TestLimitIndexLookUpKeepOrder", + "Cases": [ + { + "SQL": "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", + "Plan": [ + "Limit 0.00 root offset:0, count:10", + "└─Projection 0.00 root test.t.a, test.t.b, test.t.c, test.t.d", + " └─IndexLookUp 0.00 root ", + " ├─IndexRangeScan(Build) 2.50 cop[tikv] table:t, index:idx(a, b, c) range:(1 2,1 10), keep order:true, stats:pseudo", + " └─Selection(Probe) 0.00 cop[tikv] eq(test.t.d, 10)", + " └─TableRowIDScan 2.50 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10", + "Plan": [ + "Limit 0.00 root offset:0, count:10", + "└─Projection 0.00 root test.t.a, test.t.b, test.t.c, test.t.d", + " └─IndexLookUp 0.00 root ", + " ├─IndexRangeScan(Build) 2.50 cop[tikv] table:t, index:idx(a, b, c) range:(1 2,1 10), keep order:true, desc, stats:pseudo", + " └─Selection(Probe) 0.00 cop[tikv] eq(test.t.d, 10)", + " └─TableRowIDScan 2.50 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIssue23887", + "Cases": [ + { + "SQL": "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t", + "Plan": [ + "HashJoin 10000.00 root CARTESIAN left outer semi join, other cond:eq(2, test.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#27", + " └─Apply 10000.00 root CARTESIAN left outer join", + " ├─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#25", + " └─Limit 10000.00 root offset:0, count:1", + " └─TableReader 10000.00 root data:Limit", + " └─Limit 10000.00 cop[tikv] offset:0, count:1", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Res": [ + "1", + "1" + ] + } + ] + }, + { + "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" + ] + } + ] + }, + { + "Name": "TestIndexJoinRangeFallback", + "Cases": [ + { + "SQL": "set @@tidb_opt_range_max_size = 0", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "Plan": [ + "IndexJoin 0.50 root inner join, inner:IndexLookUp, outer key:test.t2.e, test.t2.g, inner key:test.t1.b, test.t1.d, equal cond:eq(test.t2.e, test.t1.b), eq(test.t2.g, test.t1.d)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 0.50 root ", + " ├─Selection(Build) 249.50 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", + " │ └─IndexRangeScan 250.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.b, test.t2.e) eq(test.t1.d, test.t2.g) in(test.t1.a, 1, 3) in(test.t1.c, aaa, bbb)], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.50 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", + " └─TableRowIDScan 249.50 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "set @@tidb_opt_range_max_size = 2900", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "Plan": [ + "IndexJoin 0.50 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.b, equal cond:eq(test.t2.e, test.t1.b), eq(test.t2.g, test.t1.d)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 0.50 root ", + " ├─Selection(Build) 249.50 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", + " │ └─IndexRangeScan 250.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.b, test.t2.e) in(test.t1.a, 1, 3) in(test.t1.c, aaa, bbb)], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.50 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", + " └─TableRowIDScan 249.50 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "Memory capacity of 2900 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" + ] + }, + { + "SQL": "set @@tidb_opt_range_max_size = 2300", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "Plan": [ + "IndexJoin 0.50 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.b, equal cond:eq(test.t2.e, test.t1.b), eq(test.t2.g, test.t1.d)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 0.50 root ", + " ├─Selection(Build) 249.50 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", + " │ └─IndexRangeScan 250.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.b, test.t2.e) in(test.t1.a, 1, 3)], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.50 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", + " └─TableRowIDScan 249.50 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "Memory capacity of 2300 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" + ] + }, + { + "SQL": "set @@tidb_opt_range_max_size = 700", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "Plan": [ + "HashJoin 0.05 root inner join, equal:[eq(test.t1.b, test.t2.e) eq(test.t1.d, test.t2.g)]", + "├─IndexLookUp(Build) 0.04 root ", + "│ ├─Selection(Build) 19.96 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", + "│ │ └─IndexRangeScan 20.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range:[1,1], [3,3], keep order:false, stats:pseudo", + "│ └─Selection(Probe) 0.04 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", + "│ └─TableRowIDScan 19.96 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "Memory capacity of 700 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", + "[planner:1815]Optimizer Hint /*+ INL_JOIN(t1) */ or /*+ TIDB_INLJ(t1) */ is inapplicable" + ] + }, + { + "SQL": "set @@tidb_opt_range_max_size = 0", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", + "Plan": [ + "IndexJoin 312.19 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.e))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 312.19 root ", + " ├─Selection(Build) 312.19 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 312.50 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e) gt(test.t1.b, 1) lt(test.t1.b, 10)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 312.19 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "set @@tidb_opt_range_max_size = 300", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", + "Plan": [ + "IndexJoin 312.19 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.e))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 312.19 root ", + " ├─Selection(Build) 312.19 cop[tikv] gt(test.t1.b, 1), lt(test.t1.b, 10), not(isnull(test.t1.a))", + " │ └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 312.19 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "Memory capacity of 300 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" + ] + }, + { + "SQL": "set @@tidb_opt_range_max_size = 0", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10", + "Plan": [ + "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a), other cond:gt(test.t1.b, test.t2.f), lt(test.t1.b, plus(test.t2.f, 10))", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.f))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 12475.01 root ", + " ├─Selection(Build) 12475.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e) lt(test.t1.b, plus(test.t2.f, 10)) gt(test.t1.b, test.t2.f)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "set @@tidb_opt_range_max_size = 300", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10", + "Plan": [ + "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a), other cond:gt(test.t1.b, test.t2.f), lt(test.t1.b, plus(test.t2.f, 10))", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.f))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 12475.01 root ", + " ├─Selection(Build) 12475.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "Memory capacity of 300 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" + ] + } + ] + }, + { + "Name": "TestNullConditionForPrefixIndex", + "Cases": [ + { + "SQL": "select count(1) from t1 where c1 = '0xfff' and c2 is not null", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#7)->Column#5", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo" + ], + "Result": [ + "3" + ] + }, + { + "SQL": "select count(1) from t1 where c1 = '0xfff' and c2 is null", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#5", + "└─IndexReader 0.10 root index:IndexRangeScan", + " └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" NULL,\"0xfff\" NULL], keep order:false, stats:pseudo" + ], + "Result": [ + "1" + ] + }, + { + "SQL": "select count(1) from t1 where c1 >= '0xfff' and c2 is not null", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#6)->Column#5", + "└─IndexReader 1.00 root index:HashAgg", + " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#6", + " └─Selection 3330.00 cop[tikv] not(isnull(test.t1.c2))", + " └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "3" + ] + }, + { + "SQL": "select count(1) from t1 where c1 >= '0xfff' and c2 is null", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#7)->Column#5", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#7", + " └─Selection 3.33 cop[tikv] isnull(test.t1.c2)", + " └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "1" + ] + }, + { + "SQL": "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is not null", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#5", + "└─IndexLookUp 8.00 root ", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:idx1(c1) range:[\"0xfff\",\"0xfff\"], keep order:false, stats:pseudo", + " └─Selection(Probe) 8.00 cop[tikv] not(isnull(plus(cast(test.t1.c2, double BINARY), 1)))", + " └─TableRowIDScan 10.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Result": [ + "3" + ] + }, + { + "SQL": "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is null", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#5", + "└─IndexLookUp 8.00 root ", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:idx1(c1) range:[\"0xfff\",\"0xfff\"], keep order:false, stats:pseudo", + " └─Selection(Probe) 8.00 cop[tikv] isnull(plus(cast(test.t1.c2, double BINARY), 1))", + " └─TableRowIDScan 10.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Result": [ + "1" + ] + }, + { + "SQL": "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is not null", + "Plan": [ + "Projection 99.90 root test.t1.c2", + "└─IndexLookUp 99.90 root ", + " ├─IndexRangeScan(Build) 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 99.90 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Result": [ + "", + "111111", + "22 " + ] + }, + { + "SQL": "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is null", + "Plan": [ + "Projection 0.10 root test.t1.c2", + "└─IndexLookUp 0.10 root ", + " ├─IndexRangeScan(Build) 0.10 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" NULL,\"0xfff\" NULL], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.10 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Result": [ + "" + ] + }, + { + "SQL": "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is not null", + "Plan": [ + "Projection 3330.00 root test.t1.c2", + "└─IndexLookUp 3330.00 root ", + " ├─Selection(Build) 3330.00 cop[tikv] not(isnull(test.t1.c2))", + " │ └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 3330.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Result": [ + "", + "111111", + "22 " + ] + }, + { + "SQL": "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is null", + "Plan": [ + "Projection 3.33 root test.t1.c2", + "└─IndexLookUp 3.33 root ", + " ├─Selection(Build) 3.33 cop[tikv] isnull(test.t1.c2)", + " │ └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 3.33 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Result": [ + "" + ] + }, + { + "SQL": "select count(1) from t2 use index(idx) where b is not null", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#5)->Column#4", + "└─IndexReader 1.00 root index:HashAgg", + " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#5", + " └─IndexFullScan 9990.00 cop[tikv] table:t2, index:idx(b) keep order:false, stats:pseudo" + ], + "Result": [ + "3" + ] + }, + { + "SQL": "select count(1) from t2 use index(idx) where b is null", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#6)->Column#4", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan 10.00 cop[tikv] table:t2, index:idx(b) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + "1" + ] + }, + { + "SQL": "select b from t2 use index(idx) where b is not null", + "Plan": [ + "IndexLookUp 9990.00 root ", + "├─IndexFullScan(Build) 9990.00 cop[tikv] table:t2, index:idx(b) keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 9990.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Result": [ + "", + "aaaaaa", + "bb " + ] + }, + { + "SQL": "select b from t2 use index(idx) where b is null", + "Plan": [ + "IndexLookUp 10.00 root ", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:idx(b) range:[NULL,NULL], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Result": [ + "" + ] + }, + { + "SQL": "select b from t3 where a = 1 and b is not null", + "Plan": [ + "Projection 10.00 root test.t3.b", + "└─TableReader 10.00 root data:TableRangeScan", + " └─TableRangeScan 10.00 cop[tikv] table:t3 range:[1,1], keep order:false, stats:pseudo" + ], + "Result": [ + "", + "aaaaaa", + "bb " + ] + }, + { + "SQL": "select b from t3 where a = 1 and b is null", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Result": null + } + ] + }, + { + "Name": "TestDowncastPointGetOrRangeScan", + "Cases": [ + { + "SQL": "select * from v1 where a = 1; -- the condition should be downcast through both side and go get point", + "Plan": [ + "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(test.t2.a, bigint(20) BINARY)->Column#3", + " └─Point_Get 1.00 root table:t2 handle:1" + ], + "Result": null + }, + { + "SQL": "select * from v1 where a = '1test'; -- the condition should be downcast through both side and go get point too", + "Plan": [ + "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(test.t2.a, bigint(20) BINARY)->Column#3", + " └─Point_Get 1.00 root table:t2 handle:1" + ], + "Result": null + }, + { + "SQL": "select * from v1 where a > 1; -- the condition should be downcast through both side and go range scan", + "Plan": [ + "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(test.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" + ], + "Result": null + }, + { + "SQL": "select * from v2 where a = 'test';", + "Plan": [ + "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(test.t4.a, varchar(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3", + " └─Point_Get 1.00 root table:t4, clustered index:PRIMARY(a) " + ], + "Result": null + }, + { + "SQL": "select * from v2 where a = 1;", + "Plan": [ + "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(test.t3.a, double BINARY), 1)", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─Projection 8000.00 root cast(test.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(test.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" + ], + "Result": null + }, + { + "SQL": "select * from v2 where a > 'test';", + "Plan": [ + "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(test.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" + ], + "Result": null + }, + { + "SQL": "select * from v3 where a = 'test' -- the condition shouldn't be downcast through both side and go get point", + "Plan": [ + "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(test.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(test.t6.a, char(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), \"test\")", + " └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo" + ], + "Result": null + }, + { + "SQL": "select * from v3 where a > 'test' -- the condition shouldn't be downcast through both side and go get point too", + "Plan": [ + "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(test.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(test.t6.a, char(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), \"test\")", + " └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo" + ], + "Result": null + }, + { + "SQL": "select * from v4 where a = 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range", + "Plan": [ + "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) ", + " └─Projection 8000.00 root cast(test.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3", + " └─TableReader 8000.00 root data:Selection", + " └─Selection 8000.00 cop[tikv] eq(cast(test.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), \"test\")", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Result": null + }, + { + "SQL": "select * from v4 where a > 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range", + "Plan": [ + "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", + " └─Projection 8000.00 root cast(test.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3", + " └─TableReader 8000.00 root data:Selection", + " └─Selection 8000.00 cop[tikv] gt(cast(test.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), \"test\")", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Result": null + } + ] + } +] diff --git a/planner/core/stats.go b/planner/core/stats.go index b91a9266340ed..19ca0f6a4b45a 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -418,9 +418,16 @@ func (ds *DataSource) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, ds.stats = ds.tableStats.Scale(selectivity) return ds.stats, nil } - // PushDownNot here can convert query 'not (a != 1)' to 'a = 1'. + // two preprocess here. + // 1: PushDownNot here can convert query 'not (a != 1)' to 'a = 1'. + // 2: EliminateNoPrecisionCast here can convert query 'cast(c as bigint) = 1' to 'c = 1' to leverage access range. for i, expr := range ds.pushedDownConds { +<<<<<<< HEAD ds.pushedDownConds[i] = expression.PushDownNot(ds.ctx, expr) +======= + ds.pushedDownConds[i] = expression.PushDownNot(ds.SCtx(), expr) + ds.pushedDownConds[i] = expression.EliminateNoPrecisionLossCast(ds.SCtx(), ds.pushedDownConds[i]) +>>>>>>> 28a9c7f0fb7 (planner: fix cast(col) = range couldn't build range when cast function doesn't contain any precision loss in some cases (#46303)) } for _, path := range ds.possibleAccessPaths { if path.IsTablePath() { From 65d103e2546a6f87457e5435a313ae019898217f Mon Sep 17 00:00:00 2001 From: Arenatlx Date: Tue, 31 Oct 2023 14:05:06 +0800 Subject: [PATCH 2/5] Delete planner/core/casetest/integration_test.go --- planner/core/casetest/integration_test.go | 1608 --------------------- 1 file changed, 1608 deletions(-) delete mode 100644 planner/core/casetest/integration_test.go diff --git a/planner/core/casetest/integration_test.go b/planner/core/casetest/integration_test.go deleted file mode 100644 index 9bdc5ffa7df93..0000000000000 --- a/planner/core/casetest/integration_test.go +++ /dev/null @@ -1,1608 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package casetest - -import ( - "fmt" - "strings" - "testing" - - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics/handle" - "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/testkit/testdata" - "github.com/pingcap/tidb/util" - "github.com/stretchr/testify/require" -) - -func TestAggColumnPrune(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int)") - tk.MustExec("insert into t values(1),(2)") - - var input []string - var output []struct { - SQL string - Res []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) - } -} - -func TestIsFromUnixtimeNullRejective(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(a bigint, b bigint);`) - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestSimplifyOuterJoinWithCast(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int not null, b datetime default null)") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestVerboseExplain(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec(`set tidb_opt_limit_push_down_threshold=0`) - tk.MustExec("drop table if exists t1, t2, t3") - tk.MustExec("create table t1(a int, b int)") - tk.MustExec("create table t2(a int, b int)") - tk.MustExec("create table t3(a int, b int, index c(b))") - tk.MustExec("insert into t1 values(1,2)") - tk.MustExec("insert into t1 values(3,4)") - tk.MustExec("insert into t1 values(5,6)") - tk.MustExec("insert into t2 values(1,2)") - tk.MustExec("insert into t2 values(3,4)") - tk.MustExec("insert into t2 values(5,6)") - tk.MustExec("insert into t3 values(1,2)") - tk.MustExec("insert into t3 values(3,4)") - tk.MustExec("insert into t3 values(5,6)") - tk.MustExec("analyze table t1") - tk.MustExec("analyze table t2") - tk.MustExec("analyze table t3") - - // Default RPC encoding may cause statistics explain result differ and then the test unstable. - tk.MustExec("set @@tidb_enable_chunk_rpc = on") - - // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tblInfo := range db.Tables { - if tblInfo.Name.L == "t1" || tblInfo.Name.L == "t2" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestIsolationReadTiFlashNotChoosePointGet(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int, primary key (a))") - - // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tblInfo := range db.Tables { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - - tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") - var input []string - var output []struct { - SQL string - Result []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) - } -} - -func TestIsolationReadDoNotFilterSystemDB(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("set @@tidb_isolation_read_engines = \"tiflash\"") - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestPartitionTableStats(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - { - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) - tk.MustExec("use test") - tk.MustExec(`set tidb_opt_limit_push_down_threshold=0`) - tk.MustExec("drop table if exists t") - tk.MustExec("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));") - tk.MustExec("insert into t values(21, 1), (22, 2), (23, 3), (24, 4), (15, 5)") - tk.MustExec("analyze table t") - - var input []string - var output []struct { - SQL string - Result []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) - } - } -} - -func TestPartitionPruningForInExpr(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("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);") - tk.MustExec("insert into t values (1, 1),(10, 10),(11, 11)") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestMaxMinEliminate(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int primary key)") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("create table cluster_index_t(a int, b int, c int, primary key (a, b));") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestIndexJoinUniqueCompositeIndex(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly - tk.MustExec("create table t1(a int not null, c int not null)") - tk.MustExec("create table t2(a int not null, b int not null, c int not null, primary key(a,b))") - tk.MustExec("insert into t1 values(1,1)") - tk.MustExec("insert into t2 values(1,1,1),(1,2,1)") - tk.MustExec("analyze table t1,t2") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestIndexMerge(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int, c int, unique index(a), unique index(b), primary key(c))") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -// for issue #14822 and #38258 -func TestIndexJoinTableRange(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(a int, b int, primary key (a), key idx_t1_b (b))") - tk.MustExec("create table t2(a int, b int, primary key (a), key idx_t1_b (b))") - tk.MustExec("create table t3(a int, b int, c int)") - tk.MustExec("create table t4(a int, b int, c int, primary key (a, b) clustered)") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestSubqueryWithTopN(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int)") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestApproxPercentile(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int)") - tk.MustExec("insert into t values(1, 1), (2, 1), (3, 2), (4, 2), (5, 2)") - - var input []string - var output []struct { - SQL string - Plan []string - Res []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) - output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) - } -} - -func TestIndexJoinInnerIndexNDV(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(a int not null, b int not null, c int not null)") - tk.MustExec("create table t2(a int not null, b int not null, c int not null, index idx1(a,b), index idx2(c))") - tk.MustExec("insert into t1 values(1,1,1),(1,1,1),(1,1,1)") - tk.MustExec("insert into t2 values(1,1,1),(1,1,2),(1,1,3)") - tk.MustExec("analyze table t1, t2") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestIndexMergeSerial(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a int, b int, unique key(a), unique key(b))") - tk.MustExec("insert into t value (1, 5), (2, 4), (3, 3), (4, 2), (5, 1)") - tk.MustExec("insert into t value (6, 0), (7, -1), (8, -2), (9, -3), (10, -4)") - tk.MustExec("analyze table t") - - var input []string - var output []struct { - SQL string - Plan []string - Warnings []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) - } -} - -func TestStreamAggProp(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int)") - tk.MustExec("insert into t values(1),(1),(2)") - - var input []string - var output []struct { - SQL string - Plan []string - Res []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) - } -} - -func TestIndexJoinOnClusteredIndex(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("drop table if exists t1") - tk.MustExec("create table t (a int, b varchar(20), c decimal(40,10), d int, primary key(a,b), key(c))") - tk.MustExec(`insert into t values (1,"111",1.1,11), (2,"222",2.2,12), (3,"333",3.3,13)`) - tk.MustExec("analyze table t") - - var input []string - var output []struct { - SQL string - Plan []string - Res []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery("explain format = 'brief'" + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) - } -} - -func TestPartitionExplain(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`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))`) - - tk.MustExec("set @@tidb_enable_index_merge = 1;") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) - }) - tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestIssue20710(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t;") - tk.MustExec("drop table if exists s;") - tk.MustExec("create table t(a int, b int)") - tk.MustExec("create table s(a int, b int, index(a))") - tk.MustExec("insert into t values(1,1),(1,2),(2,2)") - tk.MustExec("insert into s values(1,1),(2,2),(2,1)") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -// Apply operator may got panic because empty Projection is eliminated. -func TestIssue23887(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a int, b int);") - tk.MustExec("insert into t values(1, 2), (3, 4);") - var input []string - var output []struct { - SQL string - Plan []string - Res []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) - } - - tk.MustExec("drop table if exists t1;") - tk.MustExec("create table t1 (c1 int primary key, c2 int, c3 int, index c2 (c2));") - tk.MustQuery("select count(1) from (select count(1) from (select * from t1 where c3 = 100) k) k2;").Check(testkit.Rows("1")) -} - -func TestReorderSimplifiedOuterJoins(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists t1,t2,t3") - tk.MustExec("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))") - tk.MustExec("create table t2 (pk char(32) primary key nonclustered, col1 varchar(100))") - tk.MustExec("create table t3 (pk char(32) primary key nonclustered, keycol varchar(100), pad1 tinyint(1) default null, pad2 varchar(40), key (keycol,pad1,pad2))") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -// TestIsMatchProp is used to test https://github.com/pingcap/tidb/issues/26017. -func TestIsMatchProp(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(a int, b int, c int, d int, index idx_a_b_c(a, b, c))") - tk.MustExec("create table t2(a int, b int, c int, d int, index idx_a_b_c_d(a, b, c, d))") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestMergeContinuousSelections(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists ts") - tk.MustExec("create table ts (col_char_64 char(64), col_varchar_64_not_null varchar(64) not null, col_varchar_key varchar(1), id int primary key, col_varchar_64 varchar(64),col_char_64_not_null char(64) not null);") - - // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tblInfo := range db.Tables { - if tblInfo.Name.L == "ts" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec(" set @@tidb_allow_mpp=1;") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestLimitIndexLookUpKeepOrder(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a int, b int, c int, d int, index idx(a,b,c));") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestDecorrelateInnerJoinInSubquery(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int not null, b int not null)") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestDecorrelateLimitInSubquery(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists test") - tk.MustExec("create table test(id int, value int)") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c int)") - tk.MustExec("insert t values(10), (8), (7), (9), (11)") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestConvertRangeToPoint(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t0") - tk.MustExec("create table t0 (a int, b int, index(a, b))") - tk.MustExec("insert into t0 values (1, 1)") - tk.MustExec("insert into t0 values (2, 2)") - tk.MustExec("insert into t0 values (2, 2)") - tk.MustExec("insert into t0 values (2, 2)") - tk.MustExec("insert into t0 values (2, 2)") - tk.MustExec("insert into t0 values (2, 2)") - tk.MustExec("insert into t0 values (3, 3)") - - tk.MustExec("drop table if exists t1") - tk.MustExec("create table t1 (a int, b int, c int, index(a, b, c))") - - tk.MustExec("drop table if exists t2") - tk.MustExec("create table t2 (a float, b float, index(a, b))") - - tk.MustExec("drop table if exists t3") - tk.MustExec("create table t3 (a char(10), b char(10), c char(10), index(a, b, c))") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestIssue22105(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec(`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`) - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestRegardNULLAsPoint(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - tk.MustExec("drop table if exists tpk") - tk.MustExec(`create table tuk (a int, b int, c int, unique key (a, b, c))`) - tk.MustExec(`create table tik (a int, b int, c int, key (a, b, c))`) - for _, va := range []string{"NULL", "1"} { - for _, vb := range []string{"NULL", "1"} { - for _, vc := range []string{"NULL", "1"} { - tk.MustExec(fmt.Sprintf(`insert into tuk values (%v, %v, %v)`, va, vb, vc)) - tk.MustExec(fmt.Sprintf(`insert into tik values (%v, %v, %v)`, va, vb, vc)) - if va == "1" && vb == "1" && vc == "1" { - continue - } - // duplicated NULL rows - tk.MustExec(fmt.Sprintf(`insert into tuk values (%v, %v, %v)`, va, vb, vc)) - tk.MustExec(fmt.Sprintf(`insert into tik values (%v, %v, %v)`, va, vb, vc)) - } - } - } - - var input []string - var output []struct { - SQL string - PlanEnabled []string - PlanDisabled []string - Result []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - tk.MustExec(`set @@session.tidb_regard_null_as_point=true`) - output[i].PlanEnabled = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - - tk.MustExec(`set @@session.tidb_regard_null_as_point=false`) - output[i].PlanDisabled = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) - }) - tk.MustExec(`set @@session.tidb_regard_null_as_point=true`) - tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].PlanEnabled...)) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) - - tk.MustExec(`set @@session.tidb_regard_null_as_point=false`) - tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].PlanDisabled...)) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) - } -} - -func TestIssue30200(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t1;") - tk.MustExec("create table t1(c1 varchar(100), c2 varchar(100), key(c1), key(c2), c3 varchar(100));") - tk.MustExec("insert into t1 values('ab', '10', '10');") - - tk.MustExec("drop table if exists tt1;") - tk.MustExec("create table tt1(c1 varchar(100), c2 varchar(100), c3 varchar(100), c4 varchar(100), key idx_0(c1), key idx_1(c2, c3));") - tk.MustExec("insert into tt1 values('ab', '10', '10', '10');") - - tk.MustExec("drop table if exists tt2;") - tk.MustExec("create table tt2 (c1 int , pk int, primary key( pk ) , unique key( c1));") - tk.MustExec("insert into tt2 values(-3896405, -1), (-2, 1), (-1, -2);") - - tk.MustExec("drop table if exists tt3;") - tk.MustExec("create table tt3(c1 int, c2 int, c3 int as (c1 + c2), key(c1), key(c2), key(c3));") - tk.MustExec("insert into tt3(c1, c2) values(1, 1);") - - oriIndexMergeSwitcher := tk.MustQuery("select @@tidb_enable_index_merge;").Rows()[0][0].(string) - tk.MustExec("set tidb_enable_index_merge = on;") - defer func() { - tk.MustExec(fmt.Sprintf("set tidb_enable_index_merge = %s;", oriIndexMergeSwitcher)) - }() - - var input []string - var output []struct { - SQL string - Plan []string - Res []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format=brief " + tt).Rows()) - output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery("explain format=brief " + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) - } -} - -func TestMultiColMaxOneRow(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t1,t2") - tk.MustExec("create table t1(a int)") - tk.MustExec("create table t2(a int, b int, c int, primary key(a,b) nonclustered)") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -// TestSequenceAsDataSource is used to test https://github.com/pingcap/tidb/issues/24383. -func TestSequenceAsDataSource(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop sequence if exists s1, s2") - tk.MustExec("create sequence s1") - tk.MustExec("create sequence s2") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestHeuristicIndexSelection(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(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))") - tk.MustExec("create table t2(a int, b int, c int, d int, unique index idx_a (a), unique index idx_b_c (b, c), unique index idx_b_c_a_d (b, c, a, d))") - tk.MustExec("create table t3(a bigint, b varchar(255), c bigint, primary key(a, b) clustered)") - tk.MustExec("create table t4(a bigint, b varchar(255), c bigint, primary key(a, b) nonclustered)") - - // Default RPC encoding may cause statistics explain result differ and then the test unstable. - tk.MustExec("set @@tidb_enable_chunk_rpc = on") - - var input []string - var output []struct { - SQL string - Plan []string - Warnings []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'verbose' " + tt).Rows()) - output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) - }) - tk.MustQuery("explain format = 'verbose' " + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) - } -} - -func TestOutputSkylinePruningInfo(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists t") - tk.MustExec("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))") - - // Default RPC encoding may cause statistics explain result differ and then the test unstable. - tk.MustExec("set @@tidb_enable_chunk_rpc = on") - - var input []string - var output []struct { - SQL string - Plan []string - Warnings []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'verbose' " + tt).Rows()) - output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) - }) - tk.MustQuery("explain format = 'verbose' " + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) - } -} - -func TestPreferRangeScanForUnsignedIntHandle(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int unsigned primary key, b int, c int, index idx_b(b))") - tk.MustExec("insert into t values (1,2,3), (4,5,6), (7,8,9), (10,11,12), (13,14,15)") - do, _ := session.GetDomain(store) - require.Nil(t, do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) - tk.MustExec("analyze table t") - - // Default RPC encoding may cause statistics explain result differ and then the test unstable. - tk.MustExec("set @@tidb_enable_chunk_rpc = on") - - var input []string - var output []struct { - SQL string - Plan []string - Warnings []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - }) - if strings.HasPrefix(tt, "set") { - tk.MustExec(tt) - continue - } - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) - } -} - -func TestIssue27083(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int primary key, b int, c int, index idx_b(b))") - tk.MustExec("insert into t values (1,2,3), (4,5,6), (7,8,9), (10, 11, 12), (13,14,15), (16, 17, 18)") - do, _ := session.GetDomain(store) - require.Nil(t, do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) - tk.MustExec("analyze table t") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestGroupBySetVar(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1") - tk.MustExec("create table t1(c1 int);") - tk.MustExec("insert into t1 values(1), (2), (3), (4), (5), (6);") - rows := tk.MustQuery("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;") - rows.Check(testkit.Rows("0 2", "1 2", "2 2")) - - tk.MustExec("create table ta(a int, b int);") - tk.MustExec("set sql_mode='';") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - res := tk.MustQuery("explain format = 'brief' " + tt) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(res.Rows()) - }) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestIndexMergeWithCorrelatedColumns(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test;") - - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists t1, t2;") - tk.MustExec("create table t1(c1 int, c2 int, c3 int, primary key(c1), key(c2));") - tk.MustExec("insert into t1 values(1, 1, 1);") - tk.MustExec("insert into t1 values(2, 2, 2);") - tk.MustExec("create table t2(c1 int, c2 int, c3 int);") - tk.MustExec("insert into t2 values(1, 1, 1);") - tk.MustExec("insert into t2 values(2, 2, 2);") - - tk.MustExec("drop table if exists tt1, tt2;") - tk.MustExec("create table tt1 (c_int int, c_str varchar(40), c_datetime datetime, c_decimal decimal(12, 6), primary key(c_int), key(c_int), key(c_str), unique key(c_decimal), key(c_datetime));") - tk.MustExec("create table tt2 like tt1 ;") - tk.MustExec(`insert into tt1 (c_int, c_str, c_datetime, c_decimal) values (6, 'sharp payne', '2020-06-07 10:40:39', 6.117000) , - (7, 'objective kare', '2020-02-05 18:47:26', 1.053000) , - (8, 'thirsty pasteur', '2020-01-02 13:06:56', 2.506000) , - (9, 'blissful wilbur', '2020-06-04 11:34:04', 9.144000) , - (10, 'reverent mclean', '2020-02-12 07:36:26', 7.751000) ;`) - tk.MustExec(`insert into tt2 (c_int, c_str, c_datetime, c_decimal) values (6, 'beautiful joliot', '2020-01-16 01:44:37', 5.627000) , - (7, 'hopeful blackburn', '2020-05-23 21:44:20', 7.890000) , - (8, 'ecstatic davinci', '2020-02-01 12:27:17', 5.648000) , - (9, 'hopeful lewin', '2020-05-05 05:58:25', 7.288000) , - (10, 'sharp jennings', '2020-01-28 04:35:03', 9.758000) ;`) - - var input []string - var output []struct { - SQL string - Plan []string - Res []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format=brief " + tt).Rows()) - output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery("explain format=brief " + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) - } -} - -func TestIssue31240(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("create table t31240(a int, b int);") - tk.MustExec("set @@tidb_allow_mpp = 0") - tk.MustExec("set tidb_cost_model_version=2") - // since allow-mpp is adjusted to false, there will be no physical plan if TiFlash cop is banned. - tk.MustExec("set @@session.tidb_allow_tiflash_cop=ON") - - tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t31240", L: "t31240"}) - require.NoError(t, err) - // Set the hacked TiFlash replica for explain tests. - tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - }) - if strings.HasPrefix(tt, "set") { - tk.MustExec(tt) - continue - } - testdata.OnRecord(func() { - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } - tk.MustExec("drop table if exists t31240") -} - -func TestIssue32632(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("CREATE TABLE `partsupp` (" + - " `PS_PARTKEY` bigint(20) NOT NULL," + - "`PS_SUPPKEY` bigint(20) NOT NULL," + - "`PS_AVAILQTY` bigint(20) NOT NULL," + - "`PS_SUPPLYCOST` decimal(15,2) NOT NULL," + - "`PS_COMMENT` varchar(199) NOT NULL," + - "PRIMARY KEY (`PS_PARTKEY`,`PS_SUPPKEY`) /*T![clustered_index] NONCLUSTERED */)") - tk.MustExec("CREATE TABLE `supplier` (" + - "`S_SUPPKEY` bigint(20) NOT NULL," + - "`S_NAME` char(25) NOT NULL," + - "`S_ADDRESS` varchar(40) NOT NULL," + - "`S_NATIONKEY` bigint(20) NOT NULL," + - "`S_PHONE` char(15) NOT NULL," + - "`S_ACCTBAL` decimal(15,2) NOT NULL," + - "`S_COMMENT` varchar(101) NOT NULL," + - "PRIMARY KEY (`S_SUPPKEY`) /*T![clustered_index] CLUSTERED */)") - h := dom.StatsHandle() - require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) - tk.MustExec("set @@tidb_enforce_mpp = 1") - - tbl1, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "partsupp", L: "partsupp"}) - require.NoError(t, err) - tbl2, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "supplier", L: "supplier"}) - require.NoError(t, err) - // Set the hacked TiFlash replica for explain tests. - tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} - tbl2.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} - - statsTbl1 := h.GetTableStats(tbl1.Meta()) - statsTbl1.RealtimeCount = 800000 - statsTbl2 := h.GetTableStats(tbl2.Meta()) - statsTbl2.RealtimeCount = 10000 - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } - tk.MustExec("drop table if exists partsupp") - tk.MustExec("drop table if exists supplier") -} - -func TestTiFlashPartitionTableScan(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") - - store, dom := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=1") - tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") - tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@tidb_enforce_mpp = on") - tk.MustExec("set @@tidb_allow_batch_cop = 2") - tk.MustExec("drop table if exists rp_t;") - tk.MustExec("drop table if exists hp_t;") - tk.MustExec("create table rp_t(a int) partition by RANGE (a) (PARTITION p0 VALUES LESS THAN (6),PARTITION p1 VALUES LESS THAN (11), PARTITION p2 VALUES LESS THAN (16), PARTITION p3 VALUES LESS THAN (21));") - tk.MustExec("create table hp_t(a int) partition by hash(a) partitions 4;") - tbl1, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "rp_t", L: "rp_t"}) - require.NoError(t, err) - tbl2, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "hp_t", L: "hp_t"}) - require.NoError(t, err) - // Set the hacked TiFlash replica for explain tests. - tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} - tbl2.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } - tk.MustExec("drop table rp_t;") - tk.MustExec("drop table hp_t;") -} - -func TestTiFlashFineGrainedShuffle(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@tidb_enforce_mpp = on") - tk.MustExec("drop table if exists t1;") - tk.MustExec("create table t1(c1 int, c2 int)") - - tbl1, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t1", L: "t1"}) - require.NoError(t, err) - // Set the hacked TiFlash replica for explain tests. - tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestDowncastPointGetOrRangeScan(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t1 (a bigint key)") - tk.MustExec("create table t2 (a int key)") - tk.MustExec("create definer=`root`@`127.0.0.1` view v1 as (select a from t1) union (select a from t2)") - // select * from v where a = 1 will lead a condition: EQ(cast(t2.a as bigint), 1), - // we should downcast it, utilizing t2.a =1 to walking through the pk point-get. Because cast doesn't contain any precision loss. - - tk.MustExec("create table t3 (a varchar(100) key)") - tk.MustExec("create table t4 (a varchar(10) key)") - tk.MustExec("create definer=`root`@`127.0.0.1` view v2 as (select a from t3) union (select a from t4)") - // select * from v2 where a = 'test' will lead a condition: EQ(cast(t2.a as varchar(100) same collation), 1), - // we should downcast it, utilizing t2.a = 'test' to walking through the pk point-get. Because cast doesn't contain any precision loss. - - tk.MustExec("create table t5 (a char(100) key)") - tk.MustExec("create table t6 (a char(10) key)") - tk.MustExec("create definer=`root`@`127.0.0.1` view v3 as (select a from t5) union (select a from t6)") - // select * from v3 where a = 'test' will lead a condition: EQ(cast(t2.a as char(100) same collation), 1), - // for char type, it depends, with binary collate, the appended '0' after cast column a from char(10) to char(100) will make some difference - // on comparison on where a = 'test' before and after the UNION operator; so we didn't allow this kind of type downcast currently (precision diff). - - tk.MustExec("create table t7 (a varchar(100) key)") - tk.MustExec("create table t8 (a int key)") - tk.MustExec("create definer=`root`@`127.0.0.1` view v4 as (select a from t7) union (select a from t8)") - // since UNION OP will unify the a(int) and a(varchar100) as varchar(100) - // select * from v4 where a = "test" will lead a condition: EQ(cast(t2.a as varchar(100)), "test"), and since - // cast int to varchar(100) may have some precision loss, we couldn't utilize a="test" to get the range directly. - - var input []string - var output []struct { - SQL string - Plan []string - Result []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + tt).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) - }) - tk.MustQuery("explain format='brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Result...)) - } -} - -func TestNullConditionForPrefixIndex(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`CREATE TABLE t1 ( - id char(1) DEFAULT NULL, - c1 varchar(255) DEFAULT NULL, - c2 text DEFAULT NULL, - KEY idx1 (c1), - KEY idx2 (c1,c2(5)) -) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin`) - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("create table t2(a int, b varchar(10), index idx(b(5)))") - tk.MustExec("create table t3(a int, b varchar(10), c int, primary key (a, b(5)) clustered)") - tk.MustExec("set tidb_opt_prefix_index_single_scan = 1") - tk.MustExec("insert into t1 values ('a', '0xfff', '111111'), ('b', '0xfff', '22 '), ('c', '0xfff', ''), ('d', '0xfff', null)") - tk.MustExec("insert into t2 values (1, 'aaaaaa'), (2, 'bb '), (3, ''), (4, null)") - tk.MustExec("insert into t3 values (1, 'aaaaaa', 2), (1, 'bb ', 3), (1, '', 4)") - - var input []string - var output []struct { - SQL string - Plan []string - Result []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + tt).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) - }) - tk.MustQuery("explain format='brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Result...)) - } - - // test plan cache - tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) - tk.MustExec("set @@tidb_enable_collect_execution_info=0") - tk.MustExec("prepare stmt from 'select count(1) from t1 where c1 = ? and c2 is not null'") - tk.MustExec("set @a = '0xfff'") - tk.MustQuery("execute stmt using @a").Check(testkit.Rows("3")) - tk.MustQuery("execute stmt using @a").Check(testkit.Rows("3")) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) - tk.MustQuery("execute stmt using @a").Check(testkit.Rows("3")) - tkProcess := tk.Session().ShowProcess() - ps := []*util.ProcessInfo{tkProcess} - tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) - tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( - "StreamAgg_17 1.00 root funcs:count(Column#7)->Column#5", - "└─IndexReader_18 1.00 root index:StreamAgg_9", - " └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#7", - " └─IndexRangeScan_16 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo")) -} - -// https://github.com/pingcap/tidb/issues/24095 -func TestIssue24095(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test;") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (id int, value decimal(10,5));") - tk.MustExec("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;") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestIndexJoinRangeFallback(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(a int, b int, c varchar(10), d varchar(10), index idx_a_b_c_d(a, b, c(2), d(2)))") - tk.MustExec("create table t2(e int, f int, g varchar(10), h varchar(10))") - - var input []string - var output []struct { - SQL string - Plan []string - Warn []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - setStmt := strings.HasPrefix(tt, "set") - testdata.OnRecord(func() { - output[i].SQL = tt - if !setStmt { - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - } - }) - if setStmt { - tk.MustExec(tt) - } else { - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } - } -} - -func TestFixControl44262(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec(`use test`) - tk.MustExec(`set tidb_partition_prune_mode='dynamic'`) - tk.MustExec(`create table t1 (a int, b int)`) - tk.MustExec(`create table t2_part (a int, b int, key(a)) partition by hash(a) partitions 4`) - - testJoin := func(q, join string) { - found := false - for _, x := range tk.MustQuery(`explain ` + q).Rows() { - if strings.Contains(x[0].(string), join) { - found = true - } - } - if !found { - t.Fatal(q, join) - } - } - - testJoin(`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)`, "HashJoin") - tk.MustQuery(`show warnings`).Sort().Check(testkit.Rows( - `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`)) - tk.MustExec(`set @@tidb_opt_fix_control = "44262:ON"`) - testJoin(`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)`, "IndexJoin") - tk.MustQuery(`show warnings`).Sort().Check(testkit.Rows()) // no warning -} From 8eba3d32b2b0f073692cf0be86cee5f8e86f405b Mon Sep 17 00:00:00 2001 From: Arenatlx Date: Tue, 31 Oct 2023 14:05:34 +0800 Subject: [PATCH 3/5] Delete planner/core/casetest/testdata/integration_suite_out.json --- .../testdata/integration_suite_out.json | 4159 ----------------- 1 file changed, 4159 deletions(-) delete mode 100644 planner/core/casetest/testdata/integration_suite_out.json diff --git a/planner/core/casetest/testdata/integration_suite_out.json b/planner/core/casetest/testdata/integration_suite_out.json deleted file mode 100644 index c40e12aa02038..0000000000000 --- a/planner/core/casetest/testdata/integration_suite_out.json +++ /dev/null @@ -1,4159 +0,0 @@ -[ - { - "Name": "TestPushLimitDownIndexLookUpReader", - "Cases": [ - { - "SQL": "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 limit 2,1", - "Plan": [ - "IndexLookUp 1.00 root limit embedded(offset:2, count:1)", - "├─Limit(Build) 3.00 cop[tikv] offset:0, count:3", - "│ └─IndexRangeScan 3.00 cop[tikv] table:tbl, index:idx_b_c(b, c) range:(1,+inf], keep order:false", - "└─TableRowIDScan(Probe) 1.00 cop[tikv] table:tbl keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 order by b desc limit 2,1", - "Plan": [ - "Projection 1.00 root test.tbl.a, test.tbl.b, test.tbl.c", - "└─IndexLookUp 1.00 root limit embedded(offset:2, count:1)", - " ├─Limit(Build) 3.00 cop[tikv] offset:0, count:3", - " │ └─IndexRangeScan 3.00 cop[tikv] table:tbl, index:idx_b_c(b, c) range:(1,+inf], keep order:true, desc", - " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:tbl keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 and c > 1 limit 2,1", - "Plan": [ - "IndexLookUp 1.00 root limit embedded(offset:2, count:1)", - "├─Limit(Build) 3.00 cop[tikv] offset:0, count:3", - "│ └─Selection 3.00 cop[tikv] gt(test.tbl.c, 1)", - "│ └─IndexRangeScan 3.75 cop[tikv] table:tbl, index:idx_b_c(b, c) range:(1,+inf], keep order:false", - "└─TableRowIDScan(Probe) 1.00 cop[tikv] table:tbl keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 and a > 1 limit 2,1", - "Plan": [ - "Limit 1.00 root offset:2, count:1", - "└─IndexLookUp 3.00 root ", - " ├─IndexRangeScan(Build) 3.75 cop[tikv] table:tbl, index:idx_b_c(b, c) range:(1,+inf], keep order:false", - " └─Limit(Probe) 3.00 cop[tikv] offset:0, count:3", - " └─Selection 3.00 cop[tikv] gt(test.tbl.a, 1)", - " └─TableRowIDScan 3.75 cop[tikv] table:tbl keep order:false" - ] - } - ] - }, - { - "Name": "TestIsFromUnixtimeNullRejective", - "Cases": [ - { - "SQL": "explain format = 'brief' select * from t t1 left join t t2 on t1.a=t2.a where from_unixtime(t2.b);", - "Plan": [ - "Projection 9990.00 root test.t.a, test.t.b, test.t.a, test.t.b", - "└─HashJoin 9990.00 root inner join, equal:[eq(test.t.a, test.t.a)]", - " ├─Selection(Build) 7992.00 root from_unixtime(cast(test.t.b, decimal(20,0) BINARY))", - " │ └─TableReader 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.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(test.t.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestAggColumnPrune", - "Cases": [ - { - "SQL": "select count(1) from t join (select count(1) from t where false) as tmp", - "Res": [ - "2" - ] - }, - { - "SQL": "select count(1) from t join (select max(a) from t where false) as tmp", - "Res": [ - "2" - ] - }, - { - "SQL": "select count(1) from t join (select min(a) from t where false) as tmp", - "Res": [ - "2" - ] - }, - { - "SQL": "select count(1) from t join (select sum(a) from t where false) as tmp", - "Res": [ - "2" - ] - }, - { - "SQL": "select count(1) from t join (select avg(a) from t where false) as tmp", - "Res": [ - "2" - ] - }, - { - "SQL": "select count(1) from t join (select count(1) from t where false group by a) as tmp", - "Res": [ - "0" - ] - }, - { - "SQL": "select count(1) from t join (select max(a) from t where false group by a) as tmp", - "Res": [ - "0" - ] - }, - { - "SQL": "select count(1) from t join (select min(a) from t where false group by a) as tmp", - "Res": [ - "0" - ] - }, - { - "SQL": "select count(1) from t join (select sum(a) from t where false group by a) as tmp", - "Res": [ - "0" - ] - }, - { - "SQL": "select count(1) from t join (select avg(a) from t where false group by a) as tmp", - "Res": [ - "0" - ] - }, - { - "SQL": "SELECT avg(2) FROM(SELECT min(c) FROM t JOIN(SELECT 1 c) d ORDER BY a) e", - "Res": [ - "2.0000" - ] - } - ] - }, - { - "Name": "TestIndexJoinInnerIndexNDV", - "Cases": [ - { - "SQL": "explain format = 'brief' select /*+ inl_join(t2) */ * from t1, t2 where t1.a = t2.a and t1.b = t2.b and t1.c = t2.c", - "Plan": [ - "IndexJoin 3.00 root inner join, inner:IndexLookUp, outer key:test.t1.c, inner key:test.t2.c, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.b, test.t2.b), eq(test.t1.c, test.t2.c)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", - "└─IndexLookUp(Probe) 3.00 root ", - " ├─IndexRangeScan(Build) 3.00 cop[tikv] table:t2, index:idx2(c) range: decided by [eq(test.t2.c, test.t1.c)], keep order:false", - " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t2 keep order:false" - ] - } - ] - }, - { - "Name": "TestSimplifyOuterJoinWithCast", - "Cases": [ - { - "SQL": "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'", - "Plan": [ - "HashJoin 10000.00 root left outer join, equal:[eq(test.t.a, test.t.a)]", - "├─TableReader(Build) 8000.00 root data:Selection", - "│ └─Selection 8000.00 cop[tikv] ge(cast(test.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" - ] - } - ] - }, - { - "Name": "TestMaxMinEliminate", - "Cases": [ - { - "SQL": "explain format = 'brief' (select max(a) from t) union (select min(a) from t)", - "Plan": [ - "HashAgg 2.00 root group by:Column#5, funcs:firstrow(Column#5)->Column#5", - "└─Union 2.00 root ", - " ├─StreamAgg 1.00 root funcs:max(test.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(test.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" - ] - }, - { - "SQL": "explain format = 'brief' select min(a), max(a) from cluster_index_t", - "Plan": [ - "HashJoin 1.00 root CARTESIAN inner join", - "├─StreamAgg(Build) 1.00 root funcs:max(test.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(test.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" - ] - }, - { - "SQL": "explain format = 'brief' select min(b), max(b) from cluster_index_t where a = 1", - "Plan": [ - "HashJoin 1.00 root CARTESIAN inner join", - "├─StreamAgg(Build) 1.00 root funcs:max(test.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(test.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" - ] - }, - { - "SQL": "explain format = 'brief' select min(a), max(a) from cluster_index_t where b = 1", - "Plan": [ - "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(test.cluster_index_t.a)->Column#8, funcs:max(test.cluster_index_t.a)->Column#9", - " └─Selection 10.00 cop[tikv] eq(test.cluster_index_t.b, 1)", - " └─TableFullScan 10000.00 cop[tikv] table:cluster_index_t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select min(b), max(b) from cluster_index_t where b = 1", - "Plan": [ - "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(test.cluster_index_t.b)->Column#8, funcs:max(test.cluster_index_t.b)->Column#9", - " └─Selection 10.00 cop[tikv] eq(test.cluster_index_t.b, 1)", - " └─TableFullScan 10000.00 cop[tikv] table:cluster_index_t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestIndexJoinUniqueCompositeIndex", - "Cases": [ - { - "SQL": "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c = t2.c", - "Plan": [ - "IndexJoin 2.00 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.c, test.t2.c)", - "├─TableReader(Build) 1.00 root data:TableFullScan", - "│ └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false", - "└─IndexLookUp(Probe) 2.00 root ", - " ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false", - " └─TableRowIDScan(Probe) 2.00 cop[tikv] table:t2 keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c <= t2.b", - "Plan": [ - "IndexJoin 2.00 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a), other cond:le(test.t1.c, test.t2.b)", - "├─TableReader(Build) 1.00 root data:TableFullScan", - "│ └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false", - "└─IndexLookUp(Probe) 2.00 root ", - " ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(test.t2.a, test.t1.a) le(test.t1.c, test.t2.b)], keep order:false", - " └─TableRowIDScan(Probe) 2.00 cop[tikv] table:t2 keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t2.b = 1", - "Plan": [ - "IndexJoin 1.00 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", - "├─TableReader(Build) 1.00 root data:TableFullScan", - "│ └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false", - "└─IndexLookUp(Probe) 1.00 root ", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(test.t2.a, test.t1.a) eq(test.t2.b, 1)], keep order:false", - " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2 keep order:false" - ] - } - ] - }, - { - "Name": "TestPartitionTableStats", - "Cases": [ - { - "SQL": "explain format = 'brief' select * from t order by a", - "Result": [ - "Sort 10005.00 root test.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" - ] - }, - { - "SQL": "select * from t order by a", - "Result": [ - "15 5", - "21 1", - "22 2", - "23 3", - "24 4" - ] - }, - { - "SQL": "explain format = 'brief' select * from t order by a limit 3", - "Result": [ - "TopN 3.00 root test.t.a, offset:0, count:3", - "└─PartitionUnion 7.00 root ", - " ├─TopN 3.00 root test.t.a, offset:0, count:3", - " │ └─TableReader 3.00 root data:TopN", - " │ └─TopN 3.00 cop[tikv] test.t.a, offset:0, count:3", - " │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", - " ├─TopN 1.00 root test.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 test.t.a, offset:0, count:3", - " └─TableReader 3.00 root data:TopN", - " └─TopN 3.00 cop[tikv] test.t.a, offset:0, count:3", - " └─TableFullScan 4.00 cop[tikv] table:t, partition:p2 keep order:false" - ] - }, - { - "SQL": "select * from t order by a limit 3", - "Result": [ - "15 5", - "21 1", - "22 2" - ] - } - ] - }, - { - "Name": "TestIndexMerge", - "Cases": [ - { - "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 1 or b = 2", - "Plan": [ - "IndexMerge 2.00 root type: union", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[2,2], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, A, B) */ * from t where a = 1 or b = 2", - "Plan": [ - "IndexMerge 2.00 root type: union", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[2,2], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, primary) */ * from t where 1 or t.c", - "Plan": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b, c) */ * from t where 1 or t.a = 1 or t.b = 2", - "Plan": [ - "TableReader 10000.00 root data:Selection", - "└─Selection 10000.00 cop[tikv] or(1, or(eq(test.t.a, 1), eq(test.t.b, 2)))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestSubqueryWithTopN", - "Cases": [ - { - "SQL": "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)", - "Plan": [ - "Projection 9990.00 root test.t.b", - "└─Apply 9990.00 root semi join, equal:[eq(test.t.b, test.t.a)]", - " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.b))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - " └─Selection(Probe) 7992.00 root not(isnull(test.t.a))", - " └─Projection 9990.00 root test.t.a", - " └─TopN 9990.00 root Column#7, offset:0, count:1", - " └─Projection 9990.00 root test.t.a, plus(test.t.a, test.t.a)->Column#7", - " └─TableReader 9990.00 root data:TopN", - " └─TopN 9990.00 cop[tikv] plus(test.t.a, test.t.a), offset:0, count:1", - " └─TableFullScan 99900000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "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", - "Plan": [ - "Projection 1.00 root test.t.a", - "└─Projection 1.00 root test.t.a, test.t.b, Column#11", - " └─TopN 1.00 root Column#13, offset:0, count:1", - " └─Projection 10000.00 root test.t.a, test.t.b, Column#11, and(eq(test.t.b, 1), Column#11)->Column#13", - " └─HashJoin 10000.00 root left outer semi join, equal:[eq(test.t.b, test.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" - ] - }, - { - "SQL": "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", - "Plan": [ - "TopN 1.00 root Column#4, offset:0, count:1", - "└─Projection 10000.00 root Column#4, test.t.a, test.t.b", - " └─HashJoin 10000.00 root inner join, equal:[eq(test.t.b, Column#4)]", - " ├─Projection(Build) 8000.00 root plus(test.t.b, test.t.b)->Column#4", - " │ └─TableReader 8000.00 root data:Selection", - " │ └─Selection 8000.00 cop[tikv] not(isnull(plus(test.t.b, test.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(test.t.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestIndexJoinTableRange", - "Cases": [ - { - "SQL": "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.b", - "Plan": [ - "IndexJoin 12487.50 root inner join, inner:TableReader, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.b, test.t2.b)", - "├─IndexReader(Build) 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:idx_t1_b(b) keep order:false, stats:pseudo", - "└─TableReader(Probe) 9980.01 root data:Selection", - " └─Selection 9980.01 cop[tikv] not(isnull(test.t2.b))", - " └─TableRangeScan 9990.00 cop[tikv] table:t2 range: decided by [test.t1.a], keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.a and t1.b = t2.b", - "Plan": [ - "IndexJoin 12487.50 root inner join, inner:TableReader, outer key:test.t1.a, test.t1.b, inner key:test.t2.a, test.t2.a, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.b, test.t2.a), eq(test.t1.b, test.t2.b)", - "├─IndexReader(Build) 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:idx_t1_b(b) keep order:false, stats:pseudo", - "└─TableReader(Probe) 9980.01 root data:Selection", - " └─Selection 9980.01 cop[tikv] not(isnull(test.t2.b))", - " └─TableRangeScan 9990.00 cop[tikv] table:t2 range: decided by [test.t1.a test.t1.b], keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.a = t4.a where t4.b = 1", - "Plan": [ - "Projection 12.50 root test.t3.a, test.t3.b, test.t3.c, test.t4.a, test.t4.b, test.t4.c", - "└─IndexJoin 12.50 root inner join, inner:TableReader, outer key:test.t3.a, inner key:test.t4.a, equal cond:eq(test.t3.a, test.t4.a)", - " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9.99 root data:Selection", - " └─Selection 9.99 cop[tikv] eq(test.t4.b, 1)", - " └─TableRangeScan 9990.00 cop[tikv] table:t4 range: decided by [eq(test.t4.a, test.t3.a) eq(test.t4.b, 1)], keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.b = t4.b where t4.a = 1", - "Plan": [ - "Projection 12.50 root test.t3.a, test.t3.b, test.t3.c, test.t4.a, test.t4.b, test.t4.c", - "└─IndexJoin 12.50 root inner join, inner:TableReader, outer key:test.t3.b, inner key:test.t4.b, equal cond:eq(test.t3.b, test.t4.b)", - " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9.99 root data:Selection", - " └─Selection 9.99 cop[tikv] eq(test.t4.a, 1)", - " └─TableRangeScan 9990.00 cop[tikv] table:t4 range: decided by [eq(test.t4.b, test.t3.b) eq(test.t4.a, 1)], keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestPartitionPruningForInExpr", - "Cases": [ - { - "SQL": "explain format = 'brief' select * from t where a in (1, 2,'11')", - "Plan": [ - "TableReader 30.00 root partition:p0,p2 data:Selection", - "└─Selection 30.00 cop[tikv] in(test.t.a, 1, 2, 11)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where a in (17, null)", - "Plan": [ - "TableReader 10.00 root partition:p0,p2 data:Selection", - "└─Selection 10.00 cop[tikv] in(test.t.a, 17, NULL)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where a in (16, 'abc')", - "Plan": [ - "TableReader 20.00 root partition:p0,p2 data:Selection", - "└─Selection 20.00 cop[tikv] in(test.t.a, 16, 0)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where a in (15, 0.12, 3.47)", - "Plan": [ - "TableReader 10.00 root partition:p2 data:Selection", - "└─Selection 10.00 cop[tikv] or(eq(test.t.a, 15), 0)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where a in (0.12, 3.47)", - "Plan": [ - "TableDual 0.00 root rows:0" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where a in (14, floor(3.47))", - "Plan": [ - "TableReader 20.00 root partition:p0,p2 data:Selection", - "└─Selection 20.00 cop[tikv] in(test.t.a, 14, 3)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b in (3, 4)", - "Plan": [ - "TableReader 20.00 root partition:all data:Selection", - "└─Selection 20.00 cop[tikv] in(test.t.b, 3, 4)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestStreamAggProp", - "Cases": [ - { - "SQL": "select /*+ stream_agg() */ count(*) c from t group by a order by c limit 1", - "Plan": [ - "TopN 1.00 root Column#3, offset:0, count:1", - "└─StreamAgg 8000.00 root group by:test.t.a, funcs:count(1)->Column#3", - " └─Sort 10000.00 root test.t.a", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Res": [ - "1" - ] - }, - { - "SQL": "select /*+ stream_agg() */ count(*) c from t group by a order by c", - "Plan": [ - "Sort 8000.00 root Column#3", - "└─StreamAgg 8000.00 root group by:test.t.a, funcs:count(1)->Column#3", - " └─Sort 10000.00 root test.t.a", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Res": [ - "1", - "2" - ] - }, - { - "SQL": "select /*+ stream_agg() */ count(*) c from t group by a order by a limit 1", - "Plan": [ - "Projection 1.00 root Column#3->Column#4", - "└─Limit 1.00 root offset:0, count:1", - " └─StreamAgg 1.00 root group by:test.t.a, funcs:count(1)->Column#3, funcs:firstrow(test.t.a)->test.t.a", - " └─Sort 1.25 root test.t.a", - " └─TableReader 1.25 root data:TableFullScan", - " └─TableFullScan 1.25 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Res": [ - "2" - ] - }, - { - "SQL": "select /*+ stream_agg() */ count(*) c from t group by a order by a", - "Plan": [ - "Projection 8000.00 root Column#3->Column#4", - "└─StreamAgg 8000.00 root group by:test.t.a, funcs:count(1)->Column#3, funcs:firstrow(test.t.a)->test.t.a", - " └─Sort 10000.00 root test.t.a", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Res": [ - "2", - "1" - ] - } - ] - }, - { - "Name": "TestAccessPathOnClusterIndex", - "Cases": [ - { - "SQL": "select * from t1", - "Plan": [ - "TableReader 3.00 root data:TableFullScan", - "└─TableFullScan 3.00 cop[tikv] table:t1 keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11", - "2 222 2.2000000000 12", - "3 333 3.3000000000 13" - ] - }, - { - "SQL": "select * from t1 where t1.a >= 1 and t1.a < 4", - "Plan": [ - "TableReader 3.00 root data:TableRangeScan", - "└─TableRangeScan 3.00 cop[tikv] table:t1 range:[1,4), keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11", - "2 222 2.2000000000 12", - "3 333 3.3000000000 13" - ] - }, - { - "SQL": "select * from t1 where t1.a = 1 and t1.b < \"333\"", - "Plan": [ - "TableReader 0.82 root data:TableRangeScan", - "└─TableRangeScan 0.82 cop[tikv] table:t1 range:[1 -inf,1 \"333\"), keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11" - ] - }, - { - "SQL": "select t1.a, t1.b, t1.c from t1 where t1.c = 3.3", - "Plan": [ - "IndexReader 1.00 root index:IndexRangeScan", - "└─IndexRangeScan 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false" - ], - "Res": [ - "3 333 3.3000000000" - ] - }, - { - "SQL": "select t1.b, t1.c from t1 where t1.c = 2.2", - "Plan": [ - "IndexReader 1.00 root index:IndexRangeScan", - "└─IndexRangeScan 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false" - ], - "Res": [ - "222 2.2000000000" - ] - }, - { - "SQL": "select /*+ use_index(t1, c) */ * from t1", - "Plan": [ - "IndexLookUp 3.00 root ", - "├─IndexFullScan(Build) 3.00 cop[tikv] table:t1, index:c(c) keep order:false", - "└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t1 keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11", - "2 222 2.2000000000 12", - "3 333 3.3000000000 13" - ] - }, - { - "SQL": "select * from t1 use index(c) where t1.c in (2.2, 3.3)", - "Plan": [ - "IndexLookUp 2.00 root ", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], [3.3000000000,3.3000000000], keep order:false", - "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t1 keep order:false" - ], - "Res": [ - "2 222 2.2000000000 12", - "3 333 3.3000000000 13" - ] - }, - { - "SQL": "select * from t1 where t1.a = 1 order by b", - "Plan": [ - "TableReader 1.00 root data:TableRangeScan", - "└─TableRangeScan 1.00 cop[tikv] table:t1 range:[1,1], keep order:true" - ], - "Res": [ - "1 111 1.1000000000 11" - ] - }, - { - "SQL": "select * from t1 order by a, b limit 1", - "Plan": [ - "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:t1 keep order:true" - ], - "Res": [ - "1 111 1.1000000000 11" - ] - }, - { - "SQL": "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a >= 1 or t1.c = 2.2", - "Plan": [ - "IndexMerge 3.00 root type: union", - "├─TableRangeScan(Build) 3.00 cop[tikv] table:t1 range:[1,+inf], keep order:false", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false", - "└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t1 keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11", - "2 222 2.2000000000 12", - "3 333 3.3000000000 13" - ] - }, - { - "SQL": "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a = 1 and t1.b = '111' or t1.c = 3.3", - "Plan": [ - "IndexMerge 1.67 root type: union", - "├─TableRangeScan(Build) 1.00 cop[tikv] table:t1 range:[1 \"111\",1 \"111\"], keep order:false", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false", - "└─TableRowIDScan(Probe) 1.67 cop[tikv] table:t1 keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11", - "3 333 3.3000000000 13" - ] - } - ] - }, - { - "Name": "TestIndexJoinOnClusteredIndex", - "Cases": [ - { - "SQL": "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", - "Plan": [ - "IndexJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a, equal cond:eq(test.t.a, test.t.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", - "└─TableReader(Probe) 3.00 root data:TableRangeScan", - " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a)], keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11 1 111 1.1000000000 11", - "2 222 2.2000000000 12 2 222 2.2000000000 12", - "3 333 3.3000000000 13 3 333 3.3000000000 13" - ] - }, - { - "SQL": "select /*+ inl_merge_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", - "Plan": [ - "IndexMergeJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", - "└─TableReader(Probe) 3.00 root data:TableRangeScan", - " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a)], keep order:true" - ], - "Res": [ - "1 111 1.1000000000 11 1 111 1.1000000000 11", - "2 222 2.2000000000 12 2 222 2.2000000000 12", - "3 333 3.3000000000 13 3 333 3.3000000000 13" - ] - }, - { - "SQL": "select /*+ inl_hash_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", - "Plan": [ - "IndexHashJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a, equal cond:eq(test.t.a, test.t.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", - "└─TableReader(Probe) 3.00 root data:TableRangeScan", - " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a)], keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11 1 111 1.1000000000 11", - "2 222 2.2000000000 12 2 222 2.2000000000 12", - "3 333 3.3000000000 13 3 333 3.3000000000 13" - ] - }, - { - "SQL": "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a and t1.b = t2.b", - "Plan": [ - "IndexJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, test.t.b, inner key:test.t.a, test.t.b, equal cond:eq(test.t.a, test.t.a), eq(test.t.b, test.t.b)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", - "└─TableReader(Probe) 3.00 root data:TableRangeScan", - " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a) eq(test.t.b, test.t.b)], keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11 1 111 1.1000000000 11", - "2 222 2.2000000000 12 2 222 2.2000000000 12", - "3 333 3.3000000000 13 3 333 3.3000000000 13" - ] - }, - { - "SQL": "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.c = t2.c", - "Plan": [ - "IndexJoin 3.00 root inner join, inner:IndexLookUp, outer key:test.t.c, inner key:test.t.c, equal cond:eq(test.t.c, test.t.c)", - "├─TableReader(Build) 3.00 root data:Selection", - "│ └─Selection 3.00 cop[tikv] not(isnull(test.t.c))", - "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", - "└─IndexLookUp(Probe) 3.00 root ", - " ├─Selection(Build) 3.00 cop[tikv] not(isnull(test.t.c))", - " │ └─IndexRangeScan 3.00 cop[tikv] table:t2, index:c(c) range: decided by [eq(test.t.c, test.t.c)], keep order:false", - " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t2 keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11 1 111 1.1000000000 11", - "2 222 2.2000000000 12 2 222 2.2000000000 12", - "3 333 3.3000000000 13 3 333 3.3000000000 13" - ] - }, - { - "SQL": "select /*+ inl_merge_join(t1,t2) */ t2.a, t2.c, t2.d from t t1 left join t t2 on t1.a = t2.c;", - "Plan": [ - "IndexMergeJoin 3.00 root left outer join, inner:Projection, outer key:Column#9, inner key:test.t.c", - "├─Projection(Build) 3.00 root cast(test.t.a, decimal(10,0) BINARY)->Column#9", - "│ └─IndexReader 3.00 root index:IndexFullScan", - "│ └─IndexFullScan 3.00 cop[tikv] table:t1, index:c(c) keep order:false", - "└─Projection(Probe) 3.00 root test.t.a, test.t.c, test.t.d", - " └─IndexLookUp 3.00 root ", - " ├─IndexRangeScan(Build) 3.00 cop[tikv] table:t2, index:c(c) range: decided by [eq(test.t.c, Column#9)], keep order:true", - " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t2 keep order:false" - ], - "Res": [ - " ", - " ", - " " - ] - } - ] - }, - { - "Name": "TestPartitionExplain", - "Cases": [ - { - "SQL": "select * from pt where c > 10", - "Plan": [ - "TableReader_7 3333.33 root partition:dual data:Selection_6", - "└─Selection_6 3333.33 cop[tikv] gt(test.pt.c, 10)", - " └─TableFullScan_5 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from pt where c > 8", - "Plan": [ - "TableReader_7 3333.33 root partition:p2 data:Selection_6", - "└─Selection_6 3333.33 cop[tikv] gt(test.pt.c, 8)", - " └─TableFullScan_5 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from pt where c < 2 or c >= 9", - "Plan": [ - "TableReader_7 6656.67 root partition:p0,p2 data:Selection_6", - "└─Selection_6 6656.67 cop[tikv] or(lt(test.pt.c, 2), ge(test.pt.c, 9))", - " └─TableFullScan_5 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select c from pt", - "Plan": [ - "IndexReader_7 10000.00 root partition:all index:IndexFullScan_6", - "└─IndexFullScan_6 10000.00 cop[tikv] table:pt, index:i_c(c) keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select c from pt where c > 10", - "Plan": [ - "IndexReader_6 3333.33 root partition:dual index:IndexRangeScan_5", - "└─IndexRangeScan_5 3333.33 cop[tikv] table:pt, index:i_c(c) range:(10,+inf], keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select c from pt where c > 8", - "Plan": [ - "IndexReader_6 3333.33 root partition:p2 index:IndexRangeScan_5", - "└─IndexRangeScan_5 3333.33 cop[tikv] table:pt, index:i_c(c) range:(8,+inf], keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select c from pt where c < 2 or c >= 9", - "Plan": [ - "IndexReader_6 6656.67 root partition:p0,p2 index:IndexRangeScan_5", - "└─IndexRangeScan_5 6656.67 cop[tikv] table:pt, index:i_c(c) range:[-inf,2), [9,+inf], keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index(pt, i_id) */ * from pt", - "Plan": [ - "IndexLookUp_6 10000.00 root partition:all ", - "├─IndexFullScan_4(Build) 10000.00 cop[tikv] table:pt, index:i_id(id) keep order:false, stats:pseudo", - "└─TableRowIDScan_5(Probe) 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index(pt, i_id) */ * from pt where id < 4 and c > 10", - "Plan": [ - "IndexLookUp_8 1107.78 root partition:dual ", - "├─IndexRangeScan_5(Build) 3323.33 cop[tikv] table:pt, index:i_id(id) range:[-inf,4), keep order:false, stats:pseudo", - "└─Selection_7(Probe) 1107.78 cop[tikv] gt(test.pt.c, 10)", - " └─TableRowIDScan_6 3323.33 cop[tikv] table:pt keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c > 8", - "Plan": [ - "IndexLookUp_8 1107.78 root partition:p2 ", - "├─IndexRangeScan_5(Build) 3323.33 cop[tikv] table:pt, index:i_id(id) range:[-inf,10), keep order:false, stats:pseudo", - "└─Selection_7(Probe) 1107.78 cop[tikv] gt(test.pt.c, 8)", - " └─TableRowIDScan_6 3323.33 cop[tikv] table:pt keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c < 2 or c >= 9", - "Plan": [ - "IndexLookUp_8 5325.33 root partition:p0,p2 ", - "├─IndexFullScan_5(Build) 10000.00 cop[tikv] table:pt, index:i_id(id) keep order:false, stats:pseudo", - "└─Selection_7(Probe) 5325.33 cop[tikv] or(and(lt(test.pt.id, 10), lt(test.pt.c, 2)), ge(test.pt.c, 9))", - " └─TableRowIDScan_6 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from pt partition (p0) where c > 8", - "Plan": [ - "TableReader_7 3333.33 root partition:dual data:Selection_6", - "└─Selection_6 3333.33 cop[tikv] gt(test.pt.c, 8)", - " └─TableFullScan_5 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select c from pt partition (p0, p2) where c > 8", - "Plan": [ - "IndexReader_6 3333.33 root partition:p2 index:IndexRangeScan_5", - "└─IndexRangeScan_5 3333.33 cop[tikv] table:pt, index:i_c(c) range:(8,+inf], keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index(pt, i_id) */ * from pt partition (p1, p2) where c < 3 and id = 5", - "Plan": [ - "IndexLookUp_8 3.32 root partition:dual ", - "├─IndexRangeScan_5(Build) 10.00 cop[tikv] table:pt, index:i_id(id) range:[5,5], keep order:false, stats:pseudo", - "└─Selection_7(Probe) 3.32 cop[tikv] lt(test.pt.c, 3)", - " └─TableRowIDScan_6 10.00 cop[tikv] table:pt keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from pt where id = 4 or c < 7", - "Plan": [ - "IndexMerge_11 3330.01 root partition:all type: union", - "├─IndexRangeScan_8(Build) 10.00 cop[tikv] table:pt, index:i_id(id) range:[4,4], keep order:false, stats:pseudo", - "├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:pt, index:i_c(c) range:[-inf,7), keep order:false, stats:pseudo", - "└─TableRowIDScan_10(Probe) 3330.01 cop[tikv] table:pt keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from pt where id > 4 or c = 7", - "Plan": [ - "IndexMerge_11 3340.00 root partition:all type: union", - "├─IndexRangeScan_8(Build) 3333.33 cop[tikv] table:pt, index:i_id(id) range:(4,+inf], keep order:false, stats:pseudo", - "├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:pt, index:i_c(c) range:[7,7], keep order:false, stats:pseudo", - "└─TableRowIDScan_10(Probe) 3340.00 cop[tikv] table:pt keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestApproxPercentile", - "Cases": [ - { - "SQL": "select approx_percentile(a, 50) from t", - "Plan": [ - "HashAgg_5 1.00 root funcs:approx_percentile(test.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" - ], - "Res": [ - "3" - ] - }, - { - "SQL": "select approx_percentile(a, 10) from t", - "Plan": [ - "HashAgg_5 1.00 root funcs:approx_percentile(test.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" - ], - "Res": [ - "1" - ] - }, - { - "SQL": "select approx_percentile(a, 10+70) from t", - "Plan": [ - "HashAgg_5 1.00 root funcs:approx_percentile(test.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" - ], - "Res": [ - "4" - ] - }, - { - "SQL": "select approx_percentile(a, 10*10) from t", - "Plan": [ - "HashAgg_5 1.00 root funcs:approx_percentile(test.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" - ], - "Res": [ - "5" - ] - }, - { - "SQL": "select approx_percentile(a, 50) from t group by b order by b", - "Plan": [ - "Projection_6 8000.00 root Column#4->Column#5", - "└─Sort_7 8000.00 root test.t.b", - " └─HashAgg_9 8000.00 root group by:test.t.b, funcs:approx_percentile(test.t.a, 50)->Column#4, funcs:firstrow(test.t.b)->test.t.b", - " └─TableReader_13 10000.00 root data:TableFullScan_12", - " └─TableFullScan_12 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Res": [ - "1", - "4" - ] - } - ] - }, - { - "Name": "TestConvertRangeToPoint", - "Cases": [ - { - "SQL": "explain format = 'brief' select * from t0 where a > 1 and a < 3 order by b limit 2", - "Plan": [ - "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" - ] - }, - { - "SQL": "explain format = 'brief' select * from t1 where a >= 2 and a <= 2 and b = 2 and c > 2", - "Plan": [ - "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" - ] - }, - { - "SQL": "explain format = 'brief' select * from t2 where a >= 2.5 and a <= 2.5 order by b limit 2", - "Plan": [ - "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" - ] - }, - { - "SQL": "explain format = 'brief' select * from t3 where a >= 'a' and a <= 'a' and b = 'b' and c > 'c'", - "Plan": [ - "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" - ] - } - ] - }, - { - "Name": "TestIssue22105", - "Cases": [ - { - "SQL": "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)", - "Plan": [ - "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(test.t1.key4, 42), not(isnull(test.t1.key6))), and(eq(test.t1.key1, 4), eq(test.t1.key3, 6)))", - " └─TableRowIDScan 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestReorderSimplifiedOuterJoins", - "Cases": [ - { - "SQL": "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'", - "Plan": [ - "IndexHashJoin 13.81 root inner join, inner:IndexLookUp, outer key:test.t1.col1, inner key:test.t2.pk, equal cond:eq(test.t1.col1, test.t2.pk)", - "├─IndexHashJoin(Build) 12.50 root inner join, inner:IndexLookUp, outer key:test.t3.pk, inner key:test.t1.col3, equal cond:eq(test.t3.pk, test.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(test.t1.col3))", - "│ │ └─IndexRangeScan 18.12 cop[tikv] table:t1, index:col2(col2, col3) range: decided by [eq(test.t1.col3, test.t3.pk) eq(test.t1.col2, a)], keep order:false, stats:pseudo", - "│ └─Selection(Probe) 12.50 cop[tikv] ne(test.t1.col1, \"aaaaaa\"), ne(test.t1.col1, \"abcdef\"), not(isnull(test.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(test.t2.pk, \"aaaaaa\"), ne(test.t2.pk, \"abcdef\")", - " │ └─IndexRangeScan 12.50 cop[tikv] table:t2, index:PRIMARY(pk) range: decided by [eq(test.t2.pk, test.t1.col1)], keep order:false, stats:pseudo", - " └─Selection(Probe) 12.50 cop[tikv] in(test.t2.col1, \"a\", \"b\")", - " └─TableRowIDScan 12.50 cop[tikv] table:t2 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "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'", - "Plan": [ - "IndexHashJoin 13.81 root inner join, inner:IndexLookUp, outer key:test.t1.col1, inner key:test.t2.pk, equal cond:eq(test.t1.col1, test.t2.pk)", - "├─IndexHashJoin(Build) 12.50 root inner join, inner:IndexLookUp, outer key:test.t3.pk, inner key:test.t1.col3, equal cond:eq(test.t3.pk, test.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(test.t1.col3))", - "│ │ └─IndexRangeScan 18.12 cop[tikv] table:t1, index:col2(col2, col3) range: decided by [eq(test.t1.col3, test.t3.pk) eq(test.t1.col2, a)], keep order:false, stats:pseudo", - "│ └─Selection(Probe) 12.50 cop[tikv] ne(test.t1.col1, \"aaaaaa\"), ne(test.t1.col1, \"abcdef\"), not(isnull(test.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(test.t2.pk, \"aaaaaa\"), ne(test.t2.pk, \"abcdef\")", - " │ └─IndexRangeScan 12.50 cop[tikv] table:t2, index:PRIMARY(pk) range: decided by [eq(test.t2.pk, test.t1.col1)], keep order:false, stats:pseudo", - " └─Selection(Probe) 12.50 cop[tikv] in(test.t2.col1, \"a\", \"b\")", - " └─TableRowIDScan 12.50 cop[tikv] table:t2 keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestDecorrelateInnerJoinInSubquery", - "Cases": [ - { - "SQL": "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)", - "Plan": [ - "HashJoin 8000.00 root semi join, equal:[eq(test.t.a, test.t.a)]", - "├─HashJoin(Build) 12500.00 root inner join, equal:[eq(test.t.a, test.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" - ] - }, - { - "SQL": "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)", - "Plan": [ - "HashJoin 8000.00 root semi join, equal:[eq(test.t.a, test.t.a)]", - "├─HashJoin(Build) 12500.00 root inner join, equal:[eq(test.t.a, test.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" - ] - }, - { - "SQL": "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)", - "Plan": [ - "HashJoin 10000.00 root inner join, equal:[eq(test.t.a, test.t.a)]", - "├─HashAgg(Build) 8000.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", - "│ └─HashJoin 12500.00 root inner join, equal:[eq(test.t.a, test.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" - ] - }, - { - "SQL": "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)", - "Plan": [ - "HashJoin 10000.00 root inner join, equal:[eq(test.t.a, test.t.a)]", - "├─HashAgg(Build) 8000.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", - "│ └─HashJoin 12500.00 root inner join, equal:[eq(test.t.a, test.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" - ] - }, - { - "SQL": "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)", - "Plan": [ - "HashJoin 10000.00 root inner join, equal:[eq(test.t.a, test.t.a)]", - "├─HashAgg(Build) 8000.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", - "│ └─HashJoin 12500.00 root inner join, equal:[eq(test.t.a, test.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" - ] - }, - { - "SQL": "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)", - "Plan": [ - "HashJoin 10000.00 root inner join, equal:[eq(test.t.a, test.t.a)]", - "├─HashAgg(Build) 8000.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", - "│ └─HashJoin 12500.00 root inner join, equal:[eq(test.t.a, test.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" - ] - } - ] - }, - { - "Name": "TestDecorrelateLimitInSubquery", - "Cases": [ - { - "SQL": "explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id limit 1)", - "Plan": [ - "HashAgg 1.00 root funcs:count(1)->Column#7", - "└─HashJoin 7992.00 root semi join, equal:[eq(test.test.id, test.test.id)]", - " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.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(test.test.id))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id)", - "Plan": [ - "HashAgg 1.00 root funcs:count(1)->Column#7", - "└─HashJoin 7992.00 root semi join, equal:[eq(test.test.id, test.test.id)]", - " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.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(test.test.id))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id limit 1,2)", - "Plan": [ - "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(test.test.id, test.test.id)", - " └─TableFullScan 30000000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where 9 in (select c from t s where s.c < t.c limit 3)", - "Plan": [ - "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, test.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(test.t.c, test.t.c)", - " └─TableFullScan 37500.00 cop[tikv] table:s keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestMultiColMaxOneRow", - "Cases": [ - { - "SQL": "select (select c from t2 where t2.a = t1.a and t2.b = 1) from t1", - "Plan": [ - "HashJoin 10000.00 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", - "├─TableReader(Build) 10.00 root data:Selection", - "│ └─Selection 10.00 cop[tikv] eq(test.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" - ] - }, - { - "SQL": "select (select c from t2 where t2.a = t1.a and (t2.b = 1 or t2.b = 2)) from t1", - "Plan": [ - "Projection 10000.00 root test.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(test.t2.b, 1), eq(test.t2.b, 2))", - " │ └─IndexRangeScan 100000.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 200.00 cop[tikv] table:t2 keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestIssue24095", - "Cases": [ - { - "SQL": "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;", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#10", - "└─HashJoin 1.00 root inner join, equal:[eq(test.t.id, test.t.id) eq(test.t.value, test.t.value)]", - " ├─Selection(Build) 0.80 root not(isnull(test.t.id)), not(isnull(test.t.value))", - " │ └─TopN 1.00 root test.t.value, offset:0, count:1", - " │ └─HashJoin 12487.50 root inner join, equal:[eq(test.t.id, test.t.id)]", - " │ ├─TableReader(Build) 9990.00 root data:Selection", - " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.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(test.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(test.t.id)), not(isnull(test.t.value))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestSequenceAsDataSource", - "Cases": [ - { - "SQL": "select 1 from s1", - "Plan": [ - "Projection 1.00 root 1->Column#1", - "└─TableDual 1.00 root rows:1" - ] - }, - { - "SQL": "select count(1) from s1", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#1", - "└─TableDual 1.00 root rows:1" - ] - }, - { - "SQL": "select count(*) from s1", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#1", - "└─TableDual 1.00 root rows:1" - ] - }, - { - "SQL": "select sum(1) from s1", - "Plan": [ - "StreamAgg 1.00 root funcs:sum(1)->Column#1", - "└─TableDual 1.00 root rows:1" - ] - }, - { - "SQL": "select count(1) as cnt from s1 union select count(1) as cnt from s2", - "Plan": [ - "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" - ] - } - ] - }, - { - "Name": "TestIsMatchProp", - "Cases": [ - { - "SQL": "select a, b, c from t1 where a > 3 and b = 4 order by a, c", - "Plan": [ - "IndexReader 3.33 root index:Selection", - "└─Selection 3.33 cop[tikv] eq(test.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" - ] - }, - { - "SQL": "select * from t2 where a = 1 and c = 2 order by b, d", - "Plan": [ - "IndexReader 0.01 root index:Selection", - "└─Selection 0.01 cop[tikv] eq(test.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" - ] - }, - { - "SQL": "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", - "Plan": [ - "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" - ] - }, - { - "SQL": "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", - "Plan": [ - "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" - ] - }, - { - "SQL": "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", - "Plan": [ - "IndexReader 0.00 root index:Selection", - "└─Selection 0.00 cop[tikv] eq(test.t2.c, 3), or(and(eq(test.t2.a, 1), and(eq(test.t2.b, 1), lt(test.t2.d, 3))), and(eq(test.t2.a, 1), and(eq(test.t2.b, 1), gt(test.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" - ] - } - ] - }, - { - "Name": "TestHeuristicIndexSelection", - "Cases": [ - { - "SQL": "select * from t1 where a = 3 or a = 5", - "Plan": [ - "Batch_Point_Get_5 2.00 887.04 root table:t1 handle:[3 5], keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 handle of t1 is selected since the path only has point ranges" - ] - }, - { - "SQL": "select f, g from t1 where f = 2 and g in (3, 4, 5)", - "Plan": [ - "Batch_Point_Get_5 3.00 380.16 root table:t1, index:f_g(f, g) keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 unique index f_g of t1 is selected since the path only has point ranges with single scan" - ] - }, - { - "SQL": "select * from t1 where c = 1 and (d = 2 or d = 3) and e in (4, 5)", - "Plan": [ - "Batch_Point_Get_5 4.00 1774.08 root table:t1, index:c_d_e(c, d, e) keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 unique index c_d_e of t1 is selected since the path only has point ranges with double scan" - ] - }, - { - "SQL": "select f, g from t1 where f = 2 and g > 3", - "Plan": [ - "IndexReader_6 33.33 733.82 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 33.33 6783.33 cop[tikv] table:t1, index:f_g(f, g) range:(2 3,2 +inf], keep order:false, stats:pseudo" - ], - "Warnings": [ - "Note 1105 unique index f_g of t1 is selected since the path only fetches limited number of rows with single scan" - ] - }, - { - "SQL": "select a, b, c from t2 where a = 1 and b = 2 and c in (1, 2, 3, 4, 5)", - "Plan": [ - "Selection_6 0.01 289.88 root eq(test.t2.b, 2), in(test.t2.c, 1, 2, 3, 4, 5)", - "└─Point_Get_5 1.00 190.08 root table:t2, index:idx_a(a) " - ], - "Warnings": [ - "Note 1105 unique index idx_a of t2 is selected since the path only has point ranges with double scan" - ] - }, - { - "SQL": "select * from t3 where (a = 1 or a = 3) and b = 'xx'", - "Plan": [ - "Batch_Point_Get_5 2.00 1449.36 root table:t3, clustered index:PRIMARY(a, b) keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 handle of t3 is selected since the path only has point ranges" - ] - }, - { - "SQL": "select * from t4 where (a = 1 or a = 3) and b = 'xx'", - "Plan": [ - "Batch_Point_Get_5 2.00 1449.36 root table:t4, index:PRIMARY(a, b) keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 unique index PRIMARY of t4 is selected since the path only has point ranges with double scan" - ] - }, - { - "SQL": "select a, b from t3 where (a = 1 or a = 3) and b = 'xx'", - "Plan": [ - "Batch_Point_Get_5 2.00 1322.64 root table:t3, clustered index:PRIMARY(a, b) keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 handle of t3 is selected since the path only has point ranges" - ] - }, - { - "SQL": "select a, b from t4 where (a = 1 or a = 3) and b = 'xx'", - "Plan": [ - "Batch_Point_Get_5 2.00 1322.64 root table:t4, index:PRIMARY(a, b) keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 unique index PRIMARY of t4 is selected since the path only has point ranges with single scan" - ] - }, - { - "SQL": "update t1 set b = 2 where a = 4 or a = 6", - "Plan": [ - "Update_4 N/A N/A root N/A", - "└─Batch_Point_Get_6 2.00 887.04 root table:t1 handle:[4 6], keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 handle of t1 is selected since the path only has point ranges" - ] - }, - { - "SQL": "delete from t1 where f = 2 and g in (3, 4)", - "Plan": [ - "Delete_4 N/A N/A root N/A", - "└─Selection_7 2.00 493.42 root in(test.t1.g, 3, 4)", - " └─Point_Get_6 1.00 443.52 root table:t1, index:f(f) " - ], - "Warnings": [ - "Note 1105 unique index f of t1 is selected since the path only has point ranges with double scan" - ] - }, - { - "SQL": "insert into t3 select a, b, c from t1 where f = 2", - "Plan": [ - "Insert_1 N/A N/A root N/A", - "└─Projection_6 1.00 253.74 root test.t1.a, test.t1.b, test.t1.c", - " └─Point_Get_7 1.00 253.44 root table:t1, index:f(f) " - ], - "Warnings": [ - "Note 1105 unique index f of t1 is selected since the path only has point ranges with double scan" - ] - }, - { - "SQL": "replace into t3 select a, b, c from t1 where a = 3", - "Plan": [ - "Insert_1 N/A N/A root N/A", - "└─Point_Get_7 1.00 190.08 root table:t1 handle:3" - ], - "Warnings": [ - "Note 1105 handle of t1 is selected since the path only has point ranges" - ] - } - ] - }, - { - "Name": "TestOutputSkylinePruningInfo", - "Cases": [ - { - "SQL": "select * from t where a > 1 order by f", - "Plan": [ - "Sort_5 3333.33 2146348.14 root test.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" - ], - "Warnings": [ - "Note 1105 [t] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}", - "Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [{test.t.f asc}], TaskTp: rootTask}" - ] - }, - { - "SQL": "select * from t where f > 1", - "Plan": [ - "TableReader_7 3333.33 316532.90 root data:Selection_6", - "└─Selection_6 3333.33 3269593.45 cop[tikv] gt(test.t.f, 1)", - " └─TableFullScan_5 10000.00 2770593.45 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Warnings": [ - "Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" - ] - }, - { - "SQL": "select f from t where f > 1", - "Plan": [ - "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" - ], - "Warnings": [ - "Note 1105 [f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" - ] - }, - { - "SQL": "select * from t where f > 3 and g = 5", - "Plan": [ - "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(test.t.f, 3)", - " └─TableRowIDScan_13 10.00 2770.59 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Warnings": [ - "Note 1105 [t,f_g,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" - ] - }, - { - "SQL": "select * from t where g = 5 order by f", - "Plan": [ - "Sort_5 10.00 21321.97 root test.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" - ], - "Warnings": [ - "Note 1105 [t,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}", - "Note 1105 [t,f_g,g] remain after pruning paths for t given Prop{SortItems: [{test.t.f asc}], TaskTp: rootTask}" - ] - }, - { - "SQL": "select * from t where d = 3 order by c, e", - "Plan": [ - "IndexLookUp_15 10.00 215519.24 root ", - "├─Selection_14(Build) 10.00 2941000.00 cop[tikv] eq(test.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" - ], - "Warnings": [ - "Note 1105 [t] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}", - "Note 1105 [t,c_d_e] remain after pruning paths for t given Prop{SortItems: [{test.t.c asc} {test.t.e asc}], TaskTp: rootTask}" - ] - } - ] - }, - { - "Name": "TestPreferRangeScanForUnsignedIntHandle", - "Cases": [ - { - "SQL": "set tidb_opt_prefer_range_scan = 0", - "Plan": null, - "Warnings": null - }, - { - "SQL": "explain format = 'verbose' select * from t where b > 5", - "Plan": [ - "TableReader_7 3.00 130.42 root data:Selection_6", - "└─Selection_6 3.00 1386.04 cop[tikv] gt(test.t.b, 5)", - " └─TableFullScan_5 5.00 1136.54 cop[tikv] table:t keep order:false" - ], - "Warnings": null - }, - { - "SQL": "explain format = 'verbose' select * from t where b = 6 order by a limit 1", - "Plan": [ - "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(test.t.b, 6)", - " └─TableFullScan_21 5.00 1136.54 cop[tikv] table:t keep order:true" - ], - "Warnings": null - }, - { - "SQL": "explain format = 'verbose' select * from t where b = 6 limit 1", - "Plan": [ - "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(test.t.b, 6)", - " └─TableFullScan_10 5.00 1136.54 cop[tikv] table:t keep order:false" - ], - "Warnings": null - }, - { - "SQL": "set tidb_opt_prefer_range_scan = 1", - "Plan": null, - "Warnings": null - }, - { - "SQL": "explain format = 'verbose' select * from t where b > 5", - "Plan": [ - "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" - ], - "Warnings": [ - "Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" - ] - }, - { - "SQL": "explain format = 'verbose' select * from t where b = 6 order by a limit 1", - "Plan": [ - "TopN_9 0.00 1956.63 root test.t.a, offset:0, count:1", - "└─IndexLookUp_16 0.00 1951.83 root ", - " ├─TopN_15(Build) 0.00 206.70 cop[tikv] test.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" - ], - "Warnings": [ - "Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: copMultiReadTask}" - ] - }, - { - "SQL": "explain format = 'verbose' select * from t where b = 6 limit 1", - "Plan": [ - "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" - ], - "Warnings": [ - "Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: copMultiReadTask}" - ] - } - ] - }, - { - "Name": "TestIssue27083", - "Cases": [ - { - "SQL": "select * from t use index (idx_b) where b = 2 limit 1", - "Plan": [ - "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" - ] - } - ] - }, - { - "Name": "TestGroupBySetVar", - "Cases": [ - { - "SQL": "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;", - "Plan": [ - "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 test.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, test.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" - ] - }, - { - "SQL": "select @n:=@n+1 as e from ta group by e", - "Plan": [ - "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" - ] - }, - { - "SQL": "select @n:=@n+a as e from ta group by e", - "Plan": [ - "Projection 8000.00 root setvar(n, plus(getvar(n), cast(test.ta.a, double BINARY)))->Column#4", - "└─HashAgg 8000.00 root group by:Column#7, funcs:firstrow(Column#6)->test.ta.a", - " └─Projection 10000.00 root test.ta.a->Column#6, setvar(n, plus(getvar(n), cast(test.ta.a, double BINARY)))->Column#7", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from (select @n:=@n+1 as e from ta) tt group by e", - "Plan": [ - "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" - ] - }, - { - "SQL": "select * from (select @n:=@n+a as e from ta) tt group by e", - "Plan": [ - "HashAgg 8000.00 root group by:Column#4, funcs:firstrow(Column#4)->Column#4", - "└─Projection 10000.00 root setvar(n, plus(getvar(n), cast(test.ta.a, double BINARY)))->Column#4", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select a from ta group by @n:=@n+1", - "Plan": [ - "HashAgg 1.00 root group by:Column#5, funcs:firstrow(Column#4)->test.ta.a", - "└─Projection 10000.00 root test.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" - ] - }, - { - "SQL": "select a from ta group by @n:=@n+a", - "Plan": [ - "HashAgg 8000.00 root group by:Column#5, funcs:firstrow(Column#4)->test.ta.a", - "└─Projection 10000.00 root test.ta.a->Column#4, setvar(n, plus(getvar(n), cast(test.ta.a, double BINARY)))->Column#5", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestIssue30200", - "Cases": [ - { - "SQL": "select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab';", - "Plan": [ - "Projection 15.99 root 1->Column#5", - "└─Selection 15.99 root or(eq(test.t1.c1, \"de\"), and(eq(test.t1.c2, \"10\"), eq(from_base64(to_base64(test.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" - ], - "Res": [ - "1" - ] - }, - { - "SQL": "select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'ab' or c2 = '10' and char_length(left(c1, 10)) = 10;", - "Plan": [ - "Projection 17.99 root 1->Column#5", - "└─Selection 0.04 root or(eq(test.t1.c1, \"ab\"), and(eq(test.t1.c2, \"10\"), eq(char_length(left(test.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" - ], - "Res": [ - "1" - ] - }, - { - "SQL": "select /*+ use_index_merge(tt1) */ 1 from tt1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c3)) = '10';", - "Plan": [ - "Projection 15.99 root 1->Column#6", - "└─Selection 15.99 root or(eq(test.tt1.c1, \"de\"), and(eq(test.tt1.c2, \"10\"), eq(from_base64(to_base64(test.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" - ], - "Res": [ - "1" - ] - }, - { - "SQL": "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));", - "Plan": [ - "Projection 2.40 root 1->Column#3", - "└─Selection 2.40 root or(eq(test.tt2.c1, -3896405), and(in(test.tt2.pk, 1, 53330), istrue_with_null(cast(to_base64(left(cast(test.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" - ], - "Res": [ - "1" - ] - }, - { - "SQL": "select /*+ use_index_merge(tt3) */ 1 from tt3 where c1 < -10 or c2 < 10 and reverse(c3) = '2';", - "Plan": [ - "Projection 5098.44 root 1->Column#5", - "└─Selection 2825.66 root or(lt(test.tt3.c1, -10), and(lt(test.tt3.c2, 10), eq(reverse(cast(test.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" - ], - "Res": [ - "1" - ] - }, - { - "SQL": "select 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab';", - "Plan": [ - "Projection 8000.00 root 1->Column#5", - "└─Selection 8000.00 root or(eq(test.t1.c1, \"de\"), and(eq(test.t1.c2, \"10\"), eq(from_base64(to_base64(test.t1.c1)), \"ab\")))", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Res": [ - "1" - ] - } - ] - }, - { - "Name": "TestIndexMergeWithCorrelatedColumns", - "Cases": [ - { - "SQL": "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", - "Plan": [ - "Sort 10000.00 root test.t2.c1", - "└─Projection 10000.00 root test.t2.c1, test.t2.c2, test.t2.c3", - " └─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.t2.c1, Column#8), if(ne(Column#9, 0), NULL, 1)), or(eq(Column#10, 0), if(isnull(test.t2.c1), NULL, 0)))", - " ├─TableReader(Build) 10000.00 root data:TableFullScan", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", - " └─IndexMerge 63.35 root type: union", - " ├─Selection(Build) 10000.00 cop[tikv] eq(10, test.t2.c3)", - " │ └─TableRangeScan 10000.00 cop[tikv] table:t1 range:[10,10], keep order:false, stats:pseudo", - " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", - " │ └─IndexRangeScan 100000.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 63.35 cop[tikv] or(and(eq(test.t1.c1, 10), eq(10, test.t2.c3)), and(eq(test.t1.c2, 1), eq(1, test.t2.c3))), substring(cast(test.t1.c3, var_string(20)), 10)", - " └─TableRowIDScan 89992.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Res": [ - "1 1 1", - "2 2 2" - ] - }, - { - "SQL": "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and reverse(c3)) order by c1;", - "Plan": [ - "Sort 10000.00 root test.t2.c1", - "└─Projection 10000.00 root test.t2.c1, test.t2.c2, test.t2.c3", - " └─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.t2.c1, Column#8), if(ne(Column#9, 0), NULL, 1)), or(eq(Column#10, 0), if(isnull(test.t2.c1), NULL, 0)))", - " ├─TableReader(Build) 10000.00 root data:TableFullScan", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", - " └─IndexMerge 63.35 root type: union", - " ├─Selection(Build) 10000.00 cop[tikv] eq(10, test.t2.c3)", - " │ └─TableRangeScan 10000.00 cop[tikv] table:t1 range:[10,10], keep order:false, stats:pseudo", - " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", - " │ └─IndexRangeScan 100000.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 63.35 cop[tikv] or(and(eq(test.t1.c1, 10), eq(10, test.t2.c3)), and(eq(test.t1.c2, 1), eq(1, test.t2.c3))), reverse(cast(test.t1.c3, var_string(20)))", - " └─TableRowIDScan 89992.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Res": [ - "2 2 2" - ] - }, - { - "SQL": "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 >= 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", - "Plan": [ - "Sort 10000.00 root test.t2.c1", - "└─Projection 10000.00 root test.t2.c1, test.t2.c2, test.t2.c3", - " └─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.t2.c1, Column#8), if(ne(Column#9, 0), NULL, 1)), or(eq(Column#10, 0), if(isnull(test.t2.c1), NULL, 0)))", - " ├─TableReader(Build) 10000.00 root data:TableFullScan", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", - " └─IndexMerge 30263.46 root type: union", - " ├─Selection(Build) 33333.33 cop[tikv] eq(test.t1.c1, test.t2.c3)", - " │ └─TableRangeScan 33333333.33 cop[tikv] table:t1 range:[10,+inf], keep order:false, stats:pseudo", - " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", - " │ └─IndexRangeScan 100000.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 30263.46 cop[tikv] or(and(ge(test.t1.c1, 10), eq(test.t1.c1, test.t2.c3)), and(eq(test.t1.c2, 1), eq(1, test.t2.c3))), substring(cast(test.t1.c3, var_string(20)), 10)", - " └─TableRowIDScan 33386666.67 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Res": [ - "1 1 1", - "2 2 2" - ] - }, - { - "SQL": "select c_int from tt1 where c_decimal < all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt1.c_int = tt2.c_int and tt1.c_datetime > tt2.c_datetime and tt2.c_decimal = 9.060 or tt2.c_str <= 'interesting shtern' and tt1.c_int = tt2.c_int) order by 1;", - "Plan": [ - "Projection 10000.00 root test.tt1.c_int", - "└─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.tt1.c_decimal, Column#9), if(ne(Column#10, 0), NULL, 1)), or(eq(Column#11, 0), if(isnull(test.tt1.c_decimal), NULL, 0)))", - " ├─TableReader(Build) 10000.00 root data:TableFullScan", - " │ └─TableFullScan 10000.00 cop[tikv] table:tt1 keep order:true, stats:pseudo", - " └─StreamAgg(Probe) 10000.00 root funcs:min(Column#14)->Column#9, funcs:sum(Column#15)->Column#10, funcs:count(1)->Column#11", - " └─Projection 11.05 root test.tt2.c_decimal->Column#14, cast(isnull(test.tt2.c_decimal), decimal(20,0) BINARY)->Column#15", - " └─IndexMerge 11.05 root type: union", - " ├─Selection(Build) 10.00 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", - " │ └─IndexRangeScan 10000.00 cop[tikv] table:tt2, index:c_decimal(c_decimal) range:[9.060000,9.060000], keep order:false, stats:pseudo", - " ├─Selection(Build) 33233.33 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", - " │ └─IndexRangeScan 33233333.33 cop[tikv] table:tt2, index:c_str(c_str) range:[-inf,\"interesting shtern\"], keep order:false, stats:pseudo", - " └─Selection(Probe) 11.05 cop[tikv] or(and(eq(test.tt1.c_int, test.tt2.c_int), and(gt(test.tt1.c_datetime, test.tt2.c_datetime), eq(test.tt2.c_decimal, 9.060))), and(le(test.tt2.c_str, \"interesting shtern\"), eq(test.tt1.c_int, test.tt2.c_int)))", - " └─TableRowIDScan 33243.33 cop[tikv] table:tt2 keep order:false, stats:pseudo" - ], - "Res": [ - "7", - "8", - "10" - ] - }, - { - "SQL": "select c_int from tt1 where c_decimal > all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt2.c_int = 7 and tt2.c_int < tt1.c_decimal or tt2.c_str >= 'zzzzzzzzzzzzzzzzzzz' and tt1.c_int = tt2.c_int) order by 1;", - "Plan": [ - "Projection 10000.00 root test.tt1.c_int", - "└─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(gt(test.tt1.c_decimal, Column#9), if(ne(Column#10, 0), NULL, 1)), or(eq(Column#11, 0), if(isnull(test.tt1.c_decimal), NULL, 0)))", - " ├─TableReader(Build) 10000.00 root data:TableFullScan", - " │ └─TableFullScan 10000.00 cop[tikv] table:tt1 keep order:true, stats:pseudo", - " └─StreamAgg(Probe) 10000.00 root funcs:max(Column#14)->Column#9, funcs:sum(Column#15)->Column#10, funcs:count(1)->Column#11", - " └─Projection 17.91 root test.tt2.c_decimal->Column#14, cast(isnull(test.tt2.c_decimal), decimal(20,0) BINARY)->Column#15", - " └─IndexMerge 17.91 root type: union", - " ├─Selection(Build) 10000.00 cop[tikv] lt(7, test.tt1.c_decimal)", - " │ └─TableRangeScan 10000.00 cop[tikv] table:tt2 range:[7,7], keep order:false, stats:pseudo", - " ├─Selection(Build) 33333.33 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", - " │ └─IndexRangeScan 33333333.33 cop[tikv] table:tt2, index:c_str(c_str) range:[\"zzzzzzzzzzzzzzzzzzz\",+inf], keep order:false, stats:pseudo", - " └─Selection(Probe) 17.91 cop[tikv] or(and(eq(test.tt2.c_int, 7), lt(7, test.tt1.c_decimal)), and(ge(test.tt2.c_str, \"zzzzzzzzzzzzzzzzzzz\"), eq(test.tt1.c_int, test.tt2.c_int)))", - " └─TableRowIDScan 43330.00 cop[tikv] table:tt2 keep order:false, stats:pseudo" - ], - "Res": [ - "6", - "7", - "8", - "9" - ] - } - ] - }, - { - "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_19 3.00 162366.01 root CARTESIAN left outer semi join", - "├─Selection_36(Build) 0.80 31149.25 root eq(2, Column#18)", - "│ └─StreamAgg_43 1.00 31099.35 root funcs:count(1)->Column#18", - "│ └─TableReader_55 3.00 30949.65 root MppVersion: 2, data:ExchangeSender_54", - "│ └─ExchangeSender_54 3.00 464139.20 mpp[tiflash] ExchangeType: PassThrough", - "│ └─TableFullScan_53 3.00 464139.20 mpp[tiflash] table:t1 keep order:false", - "└─Projection_20(Probe) 3.00 129648.62 root 1->Column#28", - " └─Apply_22 3.00 129648.32 root CARTESIAN left outer join", - " ├─IndexReader_26(Build) 3.00 53.37 root index:IndexFullScan_25", - " │ └─IndexFullScan_25 3.00 610.50 cop[tikv] table:t, index:c(b) keep order:false", - " └─Projection_27(Probe) 3.00 43198.32 root 1->Column#26", - " └─Limit_30 3.00 43198.22 root offset:0, count:1", - " └─TableReader_35 3.00 43198.22 root MppVersion: 2, data:ExchangeSender_34", - " └─ExchangeSender_34 3.00 647920.44 mpp[tiflash] ExchangeType: PassThrough", - " └─Limit_33 3.00 647920.44 mpp[tiflash] offset:0, count:1", - " └─TableFullScan_32 3.00 647920.44 mpp[tiflash] table:t2 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": "TestRegardNULLAsPoint", - "Cases": [ - { - "SQL": "select * from tuk where a<=>null and b=1", - "PlanEnabled": [ - "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" - ], - "PlanDisabled": [ - "IndexReader_7 0.01 root index:Selection_6", - "└─Selection_6 0.01 cop[tikv] eq(test.tuk.b, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 ", - " 1 ", - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tik where a<=>null and b=1", - "PlanEnabled": [ - "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" - ], - "PlanDisabled": [ - "IndexReader_7 0.01 root index:Selection_6", - "└─Selection_6 0.01 cop[tikv] eq(test.tik.b, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 ", - " 1 ", - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tuk where a<=>null and b>0 and b<2", - "PlanEnabled": [ - "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" - ], - "PlanDisabled": [ - "IndexReader_7 0.25 root index:Selection_6", - "└─Selection_6 0.25 cop[tikv] eq(test.tuk.b, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 ", - " 1 ", - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tik where a<=>null and b>0 and b<2", - "PlanEnabled": [ - "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" - ], - "PlanDisabled": [ - "IndexReader_7 0.25 root index:Selection_6", - "└─Selection_6 0.25 cop[tikv] eq(test.tik.b, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 ", - " 1 ", - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tuk where a<=>null and b>=1 and b<2", - "PlanEnabled": [ - "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" - ], - "PlanDisabled": [ - "IndexReader_7 0.25 root index:Selection_6", - "└─Selection_6 0.25 cop[tikv] eq(test.tuk.b, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 ", - " 1 ", - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tik where a<=>null and b>=1 and b<2", - "PlanEnabled": [ - "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" - ], - "PlanDisabled": [ - "IndexReader_7 0.25 root index:Selection_6", - "└─Selection_6 0.25 cop[tikv] eq(test.tik.b, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 ", - " 1 ", - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tuk where a<=>null and b=1 and c=1", - "PlanEnabled": [ - "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" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] eq(test.tuk.b, 1), eq(test.tuk.c, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tik where a<=>null and b=1 and c=1", - "PlanEnabled": [ - "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" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] eq(test.tik.b, 1), eq(test.tik.c, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tuk where a=1 and b<=>null and c=1", - "PlanEnabled": [ - "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" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] eq(test.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" - ], - "Result": [ - "1 1", - "1 1" - ] - }, - { - "SQL": "select * from tik where a=1 and b<=>null and c=1", - "PlanEnabled": [ - "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" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] eq(test.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" - ], - "Result": [ - "1 1", - "1 1" - ] - }, - { - "SQL": "select * from tuk where a<=>null and b<=>null and c=1", - "PlanEnabled": [ - "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" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] eq(test.tuk.c, 1), nulleq(test.tuk.b, NULL)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1", - " 1" - ] - }, - { - "SQL": "select * from tik where a<=>null and b<=>null and c=1", - "PlanEnabled": [ - "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" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] eq(test.tik.c, 1), nulleq(test.tik.b, NULL)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1", - " 1" - ] - }, - { - "SQL": "select * from tuk where a<=>null and b<=>null and c<=>null", - "PlanEnabled": [ - "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" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] nulleq(test.tuk.b, NULL), nulleq(test.tuk.c, NULL)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " ", - " " - ] - }, - { - "SQL": "select * from tik where a<=>null and b<=>null and c<=>null", - "PlanEnabled": [ - "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" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] nulleq(test.tik.b, NULL), nulleq(test.tik.c, NULL)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " ", - " " - ] - } - ] - }, - { - "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": "TestIssue20710", - "Cases": [ - { - "SQL": "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.b", - "Plan": [ - "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.b)", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.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(test.s.a))", - " │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", - " └─Selection(Probe) 12475.01 cop[tikv] not(isnull(test.s.b))", - " └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.a", - "Plan": [ - "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.a)", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.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(test.s.a))", - " │ └─IndexRangeScan 12487.50 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:s keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.a = s.b", - "Plan": [ - "Projection 12475.01 root test.t.a, test.t.b, test.s.a, test.s.b", - "└─IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.a, test.s.b)", - " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.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(test.s.a))", - " │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", - " └─Selection(Probe) 12475.01 cop[tikv] not(isnull(test.s.b))", - " └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.b", - "Plan": [ - "IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.b)", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.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(test.s.a))", - " │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", - " └─Selection(Probe) 12475.01 cop[tikv] not(isnull(test.s.b))", - " └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.a", - "Plan": [ - "IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.a)", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.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(test.s.a))", - " │ └─IndexRangeScan 12487.50 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:s keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.a = s.b", - "Plan": [ - "Projection 12475.01 root test.t.a, test.t.b, test.s.a, test.s.b", - "└─IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.a, test.s.b)", - " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.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(test.s.a))", - " │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", - " └─Selection(Probe) 12475.01 cop[tikv] not(isnull(test.s.b))", - " └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestIndexMergeSerial", - "Cases": [ - { - "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", - "Plan": [ - "IndexMerge 8.00 root type: union", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - "├─Selection(Build) 1.00 cop[tikv] 1", - "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" - ], - "Warnings": null - }, - { - "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", - "Plan": [ - "IndexMerge 8.00 root type: union", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - "├─Selection(Build) 1.00 cop[tikv] 1", - "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" - ], - "Warnings": null - }, - { - "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", - "Plan": [ - "IndexMerge 8.00 root type: union", - "├─Selection(Build) 1.00 cop[tikv] 1", - "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - "├─Selection(Build) 1.00 cop[tikv] 1", - "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" - ], - "Warnings": null - }, - { - "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)", - "Plan": [ - "IndexMerge 0.29 root type: union", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - "└─Selection(Probe) 0.29 cop[tikv] or(and(eq(test.t.a, 1), eq(length(cast(test.t.b, var_string(20))), 1)), and(eq(test.t.b, 1), eq(length(cast(test.t.a, var_string(20))), 1)))", - " └─TableRowIDScan 1.90 cop[tikv] table:t keep order:false" - ], - "Warnings": null - } - ] - }, - { - "Name": "TestLimitIndexLookUpKeepOrder", - "Cases": [ - { - "SQL": "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", - "Plan": [ - "Limit 0.00 root offset:0, count:10", - "└─Projection 0.00 root test.t.a, test.t.b, test.t.c, test.t.d", - " └─IndexLookUp 0.00 root ", - " ├─IndexRangeScan(Build) 2.50 cop[tikv] table:t, index:idx(a, b, c) range:(1 2,1 10), keep order:true, stats:pseudo", - " └─Selection(Probe) 0.00 cop[tikv] eq(test.t.d, 10)", - " └─TableRowIDScan 2.50 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10", - "Plan": [ - "Limit 0.00 root offset:0, count:10", - "└─Projection 0.00 root test.t.a, test.t.b, test.t.c, test.t.d", - " └─IndexLookUp 0.00 root ", - " ├─IndexRangeScan(Build) 2.50 cop[tikv] table:t, index:idx(a, b, c) range:(1 2,1 10), keep order:true, desc, stats:pseudo", - " └─Selection(Probe) 0.00 cop[tikv] eq(test.t.d, 10)", - " └─TableRowIDScan 2.50 cop[tikv] table:t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestIssue23887", - "Cases": [ - { - "SQL": "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t", - "Plan": [ - "HashJoin 10000.00 root CARTESIAN left outer semi join, other cond:eq(2, test.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#27", - " └─Apply 10000.00 root CARTESIAN left outer join", - " ├─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#25", - " └─Limit 10000.00 root offset:0, count:1", - " └─TableReader 10000.00 root data:Limit", - " └─Limit 10000.00 cop[tikv] offset:0, count:1", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Res": [ - "1", - "1" - ] - } - ] - }, - { - "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" - ] - } - ] - }, - { - "Name": "TestIndexJoinRangeFallback", - "Cases": [ - { - "SQL": "set @@tidb_opt_range_max_size = 0", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "Plan": [ - "IndexJoin 0.50 root inner join, inner:IndexLookUp, outer key:test.t2.e, test.t2.g, inner key:test.t1.b, test.t1.d, equal cond:eq(test.t2.e, test.t1.b), eq(test.t2.g, test.t1.d)", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 0.50 root ", - " ├─Selection(Build) 249.50 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", - " │ └─IndexRangeScan 250.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.b, test.t2.e) eq(test.t1.d, test.t2.g) in(test.t1.a, 1, 3) in(test.t1.c, aaa, bbb)], keep order:false, stats:pseudo", - " └─Selection(Probe) 0.50 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", - " └─TableRowIDScan 249.50 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "set @@tidb_opt_range_max_size = 2900", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "Plan": [ - "IndexJoin 0.50 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.b, equal cond:eq(test.t2.e, test.t1.b), eq(test.t2.g, test.t1.d)", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 0.50 root ", - " ├─Selection(Build) 249.50 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", - " │ └─IndexRangeScan 250.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.b, test.t2.e) in(test.t1.a, 1, 3) in(test.t1.c, aaa, bbb)], keep order:false, stats:pseudo", - " └─Selection(Probe) 0.50 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", - " └─TableRowIDScan 249.50 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": [ - "Memory capacity of 2900 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" - ] - }, - { - "SQL": "set @@tidb_opt_range_max_size = 2300", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "Plan": [ - "IndexJoin 0.50 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.b, equal cond:eq(test.t2.e, test.t1.b), eq(test.t2.g, test.t1.d)", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 0.50 root ", - " ├─Selection(Build) 249.50 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", - " │ └─IndexRangeScan 250.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.b, test.t2.e) in(test.t1.a, 1, 3)], keep order:false, stats:pseudo", - " └─Selection(Probe) 0.50 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", - " └─TableRowIDScan 249.50 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": [ - "Memory capacity of 2300 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" - ] - }, - { - "SQL": "set @@tidb_opt_range_max_size = 700", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "Plan": [ - "HashJoin 0.05 root inner join, equal:[eq(test.t1.b, test.t2.e) eq(test.t1.d, test.t2.g)]", - "├─IndexLookUp(Build) 0.04 root ", - "│ ├─Selection(Build) 19.96 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", - "│ │ └─IndexRangeScan 20.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range:[1,1], [3,3], keep order:false, stats:pseudo", - "│ └─Selection(Probe) 0.04 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", - "│ └─TableRowIDScan 19.96 cop[tikv] table:t1 keep order:false, stats:pseudo", - "└─TableReader(Probe) 9980.01 root data:Selection", - " └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", - " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" - ], - "Warn": [ - "Memory capacity of 700 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", - "[planner:1815]Optimizer Hint /*+ INL_JOIN(t1) */ or /*+ TIDB_INLJ(t1) */ is inapplicable" - ] - }, - { - "SQL": "set @@tidb_opt_range_max_size = 0", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", - "Plan": [ - "IndexJoin 312.19 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a)", - "├─TableReader(Build) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.e))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 312.19 root ", - " ├─Selection(Build) 312.19 cop[tikv] not(isnull(test.t1.a))", - " │ └─IndexRangeScan 312.50 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e) gt(test.t1.b, 1) lt(test.t1.b, 10)], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 312.19 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "set @@tidb_opt_range_max_size = 300", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", - "Plan": [ - "IndexJoin 312.19 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a)", - "├─TableReader(Build) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.e))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 312.19 root ", - " ├─Selection(Build) 312.19 cop[tikv] gt(test.t1.b, 1), lt(test.t1.b, 10), not(isnull(test.t1.a))", - " │ └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e)], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 312.19 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": [ - "Memory capacity of 300 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" - ] - }, - { - "SQL": "set @@tidb_opt_range_max_size = 0", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10", - "Plan": [ - "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a), other cond:gt(test.t1.b, test.t2.f), lt(test.t1.b, plus(test.t2.f, 10))", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.f))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 12475.01 root ", - " ├─Selection(Build) 12475.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " │ └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e) lt(test.t1.b, plus(test.t2.f, 10)) gt(test.t1.b, test.t2.f)], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "set @@tidb_opt_range_max_size = 300", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10", - "Plan": [ - "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a), other cond:gt(test.t1.b, test.t2.f), lt(test.t1.b, plus(test.t2.f, 10))", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.f))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 12475.01 root ", - " ├─Selection(Build) 12475.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " │ └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e)], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": [ - "Memory capacity of 300 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" - ] - } - ] - }, - { - "Name": "TestNullConditionForPrefixIndex", - "Cases": [ - { - "SQL": "select count(1) from t1 where c1 = '0xfff' and c2 is not null", - "Plan": [ - "StreamAgg 1.00 root funcs:count(Column#7)->Column#5", - "└─IndexReader 1.00 root index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#7", - " └─IndexRangeScan 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo" - ], - "Result": [ - "3" - ] - }, - { - "SQL": "select count(1) from t1 where c1 = '0xfff' and c2 is null", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#5", - "└─IndexReader 0.10 root index:IndexRangeScan", - " └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" NULL,\"0xfff\" NULL], keep order:false, stats:pseudo" - ], - "Result": [ - "1" - ] - }, - { - "SQL": "select count(1) from t1 where c1 >= '0xfff' and c2 is not null", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#6)->Column#5", - "└─IndexReader 1.00 root index:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#6", - " └─Selection 3330.00 cop[tikv] not(isnull(test.t1.c2))", - " └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo" - ], - "Result": [ - "3" - ] - }, - { - "SQL": "select count(1) from t1 where c1 >= '0xfff' and c2 is null", - "Plan": [ - "StreamAgg 1.00 root funcs:count(Column#7)->Column#5", - "└─IndexReader 1.00 root index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#7", - " └─Selection 3.33 cop[tikv] isnull(test.t1.c2)", - " └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo" - ], - "Result": [ - "1" - ] - }, - { - "SQL": "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is not null", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#5", - "└─IndexLookUp 8.00 root ", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:idx1(c1) range:[\"0xfff\",\"0xfff\"], keep order:false, stats:pseudo", - " └─Selection(Probe) 8.00 cop[tikv] not(isnull(plus(cast(test.t1.c2, double BINARY), 1)))", - " └─TableRowIDScan 10.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Result": [ - "3" - ] - }, - { - "SQL": "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is null", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#5", - "└─IndexLookUp 8.00 root ", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:idx1(c1) range:[\"0xfff\",\"0xfff\"], keep order:false, stats:pseudo", - " └─Selection(Probe) 8.00 cop[tikv] isnull(plus(cast(test.t1.c2, double BINARY), 1))", - " └─TableRowIDScan 10.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Result": [ - "1" - ] - }, - { - "SQL": "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is not null", - "Plan": [ - "Projection 99.90 root test.t1.c2", - "└─IndexLookUp 99.90 root ", - " ├─IndexRangeScan(Build) 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 99.90 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Result": [ - "", - "111111", - "22 " - ] - }, - { - "SQL": "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is null", - "Plan": [ - "Projection 0.10 root test.t1.c2", - "└─IndexLookUp 0.10 root ", - " ├─IndexRangeScan(Build) 0.10 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" NULL,\"0xfff\" NULL], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.10 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Result": [ - "" - ] - }, - { - "SQL": "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is not null", - "Plan": [ - "Projection 3330.00 root test.t1.c2", - "└─IndexLookUp 3330.00 root ", - " ├─Selection(Build) 3330.00 cop[tikv] not(isnull(test.t1.c2))", - " │ └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 3330.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Result": [ - "", - "111111", - "22 " - ] - }, - { - "SQL": "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is null", - "Plan": [ - "Projection 3.33 root test.t1.c2", - "└─IndexLookUp 3.33 root ", - " ├─Selection(Build) 3.33 cop[tikv] isnull(test.t1.c2)", - " │ └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 3.33 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Result": [ - "" - ] - }, - { - "SQL": "select count(1) from t2 use index(idx) where b is not null", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#5)->Column#4", - "└─IndexReader 1.00 root index:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#5", - " └─IndexFullScan 9990.00 cop[tikv] table:t2, index:idx(b) keep order:false, stats:pseudo" - ], - "Result": [ - "3" - ] - }, - { - "SQL": "select count(1) from t2 use index(idx) where b is null", - "Plan": [ - "StreamAgg 1.00 root funcs:count(Column#6)->Column#4", - "└─IndexReader 1.00 root index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#6", - " └─IndexRangeScan 10.00 cop[tikv] table:t2, index:idx(b) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - "1" - ] - }, - { - "SQL": "select b from t2 use index(idx) where b is not null", - "Plan": [ - "IndexLookUp 9990.00 root ", - "├─IndexFullScan(Build) 9990.00 cop[tikv] table:t2, index:idx(b) keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 9990.00 cop[tikv] table:t2 keep order:false, stats:pseudo" - ], - "Result": [ - "", - "aaaaaa", - "bb " - ] - }, - { - "SQL": "select b from t2 use index(idx) where b is null", - "Plan": [ - "IndexLookUp 10.00 root ", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:idx(b) range:[NULL,NULL], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo" - ], - "Result": [ - "" - ] - }, - { - "SQL": "select b from t3 where a = 1 and b is not null", - "Plan": [ - "Projection 10.00 root test.t3.b", - "└─TableReader 10.00 root data:TableRangeScan", - " └─TableRangeScan 10.00 cop[tikv] table:t3 range:[1,1], keep order:false, stats:pseudo" - ], - "Result": [ - "", - "aaaaaa", - "bb " - ] - }, - { - "SQL": "select b from t3 where a = 1 and b is null", - "Plan": [ - "TableDual 0.00 root rows:0" - ], - "Result": null - } - ] - }, - { - "Name": "TestDowncastPointGetOrRangeScan", - "Cases": [ - { - "SQL": "select * from v1 where a = 1; -- the condition should be downcast through both side and go get point", - "Plan": [ - "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(test.t2.a, bigint(20) BINARY)->Column#3", - " └─Point_Get 1.00 root table:t2 handle:1" - ], - "Result": null - }, - { - "SQL": "select * from v1 where a = '1test'; -- the condition should be downcast through both side and go get point too", - "Plan": [ - "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(test.t2.a, bigint(20) BINARY)->Column#3", - " └─Point_Get 1.00 root table:t2 handle:1" - ], - "Result": null - }, - { - "SQL": "select * from v1 where a > 1; -- the condition should be downcast through both side and go range scan", - "Plan": [ - "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(test.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" - ], - "Result": null - }, - { - "SQL": "select * from v2 where a = 'test';", - "Plan": [ - "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(test.t4.a, varchar(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3", - " └─Point_Get 1.00 root table:t4, clustered index:PRIMARY(a) " - ], - "Result": null - }, - { - "SQL": "select * from v2 where a = 1;", - "Plan": [ - "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(test.t3.a, double BINARY), 1)", - " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - " └─Projection 8000.00 root cast(test.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(test.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" - ], - "Result": null - }, - { - "SQL": "select * from v2 where a > 'test';", - "Plan": [ - "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(test.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" - ], - "Result": null - }, - { - "SQL": "select * from v3 where a = 'test' -- the condition shouldn't be downcast through both side and go get point", - "Plan": [ - "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(test.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(test.t6.a, char(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), \"test\")", - " └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo" - ], - "Result": null - }, - { - "SQL": "select * from v3 where a > 'test' -- the condition shouldn't be downcast through both side and go get point too", - "Plan": [ - "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(test.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(test.t6.a, char(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), \"test\")", - " └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo" - ], - "Result": null - }, - { - "SQL": "select * from v4 where a = 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range", - "Plan": [ - "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) ", - " └─Projection 8000.00 root cast(test.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3", - " └─TableReader 8000.00 root data:Selection", - " └─Selection 8000.00 cop[tikv] eq(cast(test.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), \"test\")", - " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" - ], - "Result": null - }, - { - "SQL": "select * from v4 where a > 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range", - "Plan": [ - "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", - " └─Projection 8000.00 root cast(test.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3", - " └─TableReader 8000.00 root data:Selection", - " └─Selection 8000.00 cop[tikv] gt(cast(test.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), \"test\")", - " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" - ], - "Result": null - } - ] - } -] From b14dcb5297c5c415b5139d2595a228a2b3a48b46 Mon Sep 17 00:00:00 2001 From: Arenatlx Date: Tue, 31 Oct 2023 14:05:52 +0800 Subject: [PATCH 4/5] Delete planner/core/casetest/testdata/integration_suite_in.json --- .../testdata/integration_suite_in.json | 613 ------------------ 1 file changed, 613 deletions(-) delete mode 100644 planner/core/casetest/testdata/integration_suite_in.json diff --git a/planner/core/casetest/testdata/integration_suite_in.json b/planner/core/casetest/testdata/integration_suite_in.json deleted file mode 100644 index 9a4d4801a4b73..0000000000000 --- a/planner/core/casetest/testdata/integration_suite_in.json +++ /dev/null @@ -1,613 +0,0 @@ -[ - { - "name": "TestPushLimitDownIndexLookUpReader", - "cases": [ - // Limit should be pushed down into IndexLookUpReader, row count of IndexLookUpReader and TableScan should be 1.00. - "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 limit 2,1", - // Projection atop IndexLookUpReader, Limit should be pushed down into IndexLookUpReader, and Projection should have row count 1.00 as well. - "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 order by b desc limit 2,1", - // Limit should be pushed down into IndexLookUpReader when Selection on top of IndexScan. - "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 and c > 1 limit 2,1", - // Limit should NOT be pushed down into IndexLookUpReader when Selection on top of TableScan. - "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 and a > 1 limit 2,1" - ] - }, - { - "name": "TestIsFromUnixtimeNullRejective", - "cases": [ - // fix #12385 - "explain format = 'brief' select * from t t1 left join t t2 on t1.a=t2.a where from_unixtime(t2.b);" - ] - }, - { - "name": "TestAggColumnPrune", - "cases": [ - "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" - ] - }, - { - "name": "TestIndexJoinInnerIndexNDV", - "cases": [ - // t2 should use idx2 instead of idx1, since idx2 has larger NDV. - "explain format = 'brief' select /*+ inl_join(t2) */ * from t1, t2 where t1.a = t2.a and t1.b = t2.b and t1.c = t2.c" - ] - }, - { - "name": "TestSimplifyOuterJoinWithCast", - "cases": [ - // LeftOuterJoin should no be simplified to InnerJoin. - "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'" - ] - }, - { - "name": "TestMaxMinEliminate", - "cases": [ - "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" - ] - }, - { - "name": "TestIndexJoinUniqueCompositeIndex", - "cases": [ - // Row count of IndexScan should be 2. - "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c = t2.c", - // Row count of IndexScan should be 2. - "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c <= t2.b", - // Row count of IndexScan should be 1. - "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t2.b = 1" - ] - }, - { - "name": "TestPartitionTableStats", - "cases": [ - "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" - ] - }, - { - "name": "TestIndexMerge", - "cases": [ - "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 1 or b = 2", - "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, A, B) */ * from t where a = 1 or b = 2", - "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, primary) */ * from t where 1 or t.c", - "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b, c) */ * from t where 1 or t.a = 1 or t.b = 2" - ] - }, - { - "name": "TestSubqueryWithTopN", - "cases": [ - "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" - ] - }, - { - "name": "TestIndexJoinTableRange", - "cases": [ - "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.b", - "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.a and t1.b = t2.b", - "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.a = t4.a where t4.b = 1", - "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.b = t4.b where t4.a = 1" - ] - }, - { - "name": "TestPartitionPruningForInExpr", - "cases": [ - "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)" - ] - }, - { - "name": "TestStreamAggProp", - "cases": [ - "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", - "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" - ] - }, - { - "name": "TestAccessPathOnClusterIndex", - "cases": [ - "select * from t1", - "select * from t1 where t1.a >= 1 and t1.a < 4", - "select * from t1 where t1.a = 1 and t1.b < \"333\"", - "select t1.a, t1.b, t1.c from t1 where t1.c = 3.3", - "select t1.b, t1.c from t1 where t1.c = 2.2", - "select /*+ use_index(t1, c) */ * from t1", - "select * from t1 use index(c) where t1.c in (2.2, 3.3)", - "select * from t1 where t1.a = 1 order by b", - "select * from t1 order by a, b limit 1", - "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a >= 1 or t1.c = 2.2", - "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a = 1 and t1.b = '111' or t1.c = 3.3" - ] - }, - { - "name": "TestIndexJoinOnClusteredIndex", - "cases": [ - "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", - "select /*+ inl_merge_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", - "select /*+ inl_hash_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", - "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a and t1.b = t2.b", - "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.c = t2.c", - "select /*+ inl_merge_join(t1,t2) */ t2.a, t2.c, t2.d from t t1 left join t t2 on t1.a = t2.c;" - ] - }, - { - "name": "TestPartitionExplain", - "cases": [ - // Table reader - "select * from pt where c > 10", - "select * from pt where c > 8", - "select * from pt where c < 2 or c >= 9", - // Index reader - "select c from pt", - "select c from pt where c > 10", - "select c from pt where c > 8", - "select c from pt where c < 2 or c >= 9", - // Index Lookup - "select /*+ use_index(pt, i_id) */ * from pt", - "select /*+ use_index(pt, i_id) */ * from pt where id < 4 and c > 10", - "select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c > 8", - "select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c < 2 or c >= 9", - // Partition selection - "select * from pt partition (p0) where c > 8", - "select c from pt partition (p0, p2) where c > 8", - "select /*+ use_index(pt, i_id) */ * from pt partition (p1, p2) where c < 3 and id = 5", - // Index Merge - "select * from pt where id = 4 or c < 7", - "select * from pt where id > 4 or c = 7" - ] - }, - { - "name": "TestApproxPercentile", - "cases": [ - "select approx_percentile(a, 50) from t", - "select approx_percentile(a, 10) from t", - "select approx_percentile(a, 10+70) from t", - "select approx_percentile(a, 10*10) from t", - "select approx_percentile(a, 50) from t group by b order by b" - ] - }, - { - "name": "TestConvertRangeToPoint", - "cases": [ - "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'" - ] - }, - { - "name": "TestIssue22105", - "cases": [ - "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)" - ] - }, - { - "name": "TestReorderSimplifiedOuterJoins", - "cases": [ - // Query with INNER JOIN or LEFT JOIN should have the same plan. - "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'" - ] - }, - { - "name": "TestDecorrelateInnerJoinInSubquery", - "cases": [ - // Query with WHERE or ON should have the same plan, i.e, the Apply has been decorrelated. - "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)" - ] - }, - { - "name": "TestDecorrelateLimitInSubquery", - "cases": [ - // Query with EXISTS and subquery with LIMIT should have the same plan, i.e, the Limit has been decorrelated. - "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)" - ] - }, - { - "name": "TestMultiColMaxOneRow", - "cases": [ - "select (select c from t2 where t2.a = t1.a and t2.b = 1) from t1", - "select (select c from t2 where t2.a = t1.a and (t2.b = 1 or t2.b = 2)) from t1" - ] - }, - { - "name": "TestIssue24095", - "cases": [ - "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;" - ] - }, - { - "name": "TestSequenceAsDataSource", - "cases": [ - "select 1 from s1", - "select count(1) from s1", - "select count(*) from s1", - "select sum(1) from s1", - "select count(1) as cnt from s1 union select count(1) as cnt from s2" - ] - }, - { - "name": "TestIsMatchProp", - "cases": [ - "select a, b, c from t1 where a > 3 and b = 4 order by a, c", - "select * from t2 where a = 1 and c = 2 order by b, d", - "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", - "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", - "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" - ] - }, - { - "name": "TestHeuristicIndexSelection", - "cases": [ - "select * from t1 where a = 3 or a = 5", - "select f, g from t1 where f = 2 and g in (3, 4, 5)", - "select * from t1 where c = 1 and (d = 2 or d = 3) and e in (4, 5)", - "select f, g from t1 where f = 2 and g > 3", - "select a, b, c from t2 where a = 1 and b = 2 and c in (1, 2, 3, 4, 5)", - "select * from t3 where (a = 1 or a = 3) and b = 'xx'", - "select * from t4 where (a = 1 or a = 3) and b = 'xx'", - "select a, b from t3 where (a = 1 or a = 3) and b = 'xx'", - "select a, b from t4 where (a = 1 or a = 3) and b = 'xx'", - "update t1 set b = 2 where a = 4 or a = 6", - "delete from t1 where f = 2 and g in (3, 4)", - "insert into t3 select a, b, c from t1 where f = 2", - "replace into t3 select a, b, c from t1 where a = 3" - ] - }, - { - "name": "TestOutputSkylinePruningInfo", - "cases": [ - "select * from t where a > 1 order by f", - "select * from t where f > 1", - "select f from t where f > 1", - "select * from t where f > 3 and g = 5", - "select * from t where g = 5 order by f", - "select * from t where d = 3 order by c, e" - ] - }, - { - "name": "TestPreferRangeScanForUnsignedIntHandle", - "cases": [ - "set tidb_opt_prefer_range_scan = 0", - "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" - ] - }, - { - "name": "TestIssue27083", - "cases": [ - "select * from t use index (idx_b) where b = 2 limit 1" - ] - }, - { - "name": "TestGroupBySetVar", - "cases": [ - "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;", - // TODO: fix these two cases - "select @n:=@n+1 as e from ta group by e", - "select @n:=@n+a as e from ta group by e", - "select * from (select @n:=@n+1 as e from ta) tt group by e", - "select * from (select @n:=@n+a as e from ta) tt group by e", - "select a from ta group by @n:=@n+1", - "select a from ta group by @n:=@n+a" - ] - }, - { - "name": "TestIssue30200", - "cases": [ - // to_base64 and from_base64 has not been pushed to TiKV or TiFlash. - // We expect a Selection will be added above IndexMerge. - "select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab';", - // `left` has not been pushed to TiKV, but it has been pushed to TiFlash. - // We expect a Selection will be added above IndexMerge. - "select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'ab' or c2 = '10' and char_length(left(c1, 10)) = 10;", - // c3 is part of idx_1, so it will be put in partial_path's IndexFilters instead of TableFilters. - // But it still cannot be pushed to TiKV. This case cover code in DataSource.buildIndexMergeOrPath. - "select /*+ use_index_merge(tt1) */ 1 from tt1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c3)) = '10';", - // to_base64(left(pk, 5)) is in partial_path's TableFilters. But it cannot be pushed to TiKV. - // So it should be executed in TiDB. This case cover code in DataSource.buildIndexMergeOrPath. - "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));", - // This case covert expression index. - "select /*+ use_index_merge(tt3) */ 1 from tt3 where c1 < -10 or c2 < 10 and reverse(c3) = '2';", - // If no hint, we cannot use index merge if filter cannot be pushed to any storage. - "select 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab';" - ] - }, - { - "name": "TestIndexMergeWithCorrelatedColumns", - "cases": [ - "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", - "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and reverse(c3)) order by c1;", - "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 >= 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", - // Test correlated column in IndexPath.TableFilters. - "select c_int from tt1 where c_decimal < all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt1.c_int = tt2.c_int and tt1.c_datetime > tt2.c_datetime and tt2.c_decimal = 9.060 or tt2.c_str <= 'interesting shtern' and tt1.c_int = tt2.c_int) order by 1;", - // Test correlated column in TablePath.TableFilters. - "select c_int from tt1 where c_decimal > all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt2.c_int = 7 and tt2.c_int < tt1.c_decimal or tt2.c_str >= 'zzzzzzzzzzzzzzzzzzz' and tt1.c_int = tt2.c_int) order by 1;" - ] - }, - { - "name": "TestIssue31240", - "cases": [ - "explain format = 'brief' select count(*) from t31240;", - "set @@tidb_isolation_read_engines=\"tiflash,tidb\";", - "explain format = 'brief' select count(*) from t31240;" - ] - }, - { - "name": "TestVerboseExplain", - "cases": [ - "explain format = 'verbose' select count(*) from t3", - "explain format = 'verbose' select count(*) from t2", - "explain format = 'verbose' select * from t3 order by a", - "explain format = 'verbose' select * from t3 order by b", - "explain format = 'verbose' select * from t3 order by a limit 1", - "explain format = 'verbose' select * from t3 order by b limit 1", - "explain format = 'verbose' select count(*) from t2 group by a", - "explain format = 'verbose' select count(*) from t3 where b = 0", - "explain format = 'verbose' select /*+ use_index(t3, c) */ count(a) from t3 where b = 0", - "explain format = 'verbose' select count(*) from t2 where a = 0", - "explain format = 'verbose' select count(*) from t3 t join t3 on t.a = t3.b", - "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", - "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", - "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", - "explain format = 'verbose' select /*+ merge_join(t1), read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a" - ] - }, - { - "name": "TestRegardNULLAsPoint", - "cases": [ - "select * from tuk where a<=>null and b=1", - "select * from tik where a<=>null and b=1", - "select * from tuk where a<=>null and b>0 and b<2", - "select * from tik where a<=>null and b>0 and b<2", - "select * from tuk where a<=>null and b>=1 and b<2", - "select * from tik where a<=>null and b>=1 and b<2", - "select * from tuk where a<=>null and b=1 and c=1", - "select * from tik where a<=>null and b=1 and c=1", - "select * from tuk where a=1 and b<=>null and c=1", - "select * from tik where a=1 and b<=>null and c=1", - "select * from tuk where a<=>null and b<=>null and c=1", - "select * from tik where a<=>null and b<=>null and c=1", - "select * from tuk where a<=>null and b<=>null and c<=>null", - "select * from tik where a<=>null and b<=>null and c<=>null" - ] - }, - { - "name": "TestIsolationReadDoNotFilterSystemDB", - "cases": [ - "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'", - "desc format = 'brief' select * from information_schema.tables", - "desc format = 'brief' select * from mysql.stats_meta" - ] - }, - { - "name": "TestIsolationReadTiFlashNotChoosePointGet", - "cases": [ - "explain format = 'brief' select * from t where t.a = 1", - "explain format = 'brief' select * from t where t.a in (1, 2)" - ] - }, - { - "name": "TestIssue20710", - "cases": [ - "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" - ] - }, - { - "name": "TestIndexMergeSerial", - "cases": [ - "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", - "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", - "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", - "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)" - ] - }, - { - "name": "TestLimitIndexLookUpKeepOrder", - "cases": [ - "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", - "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10" - ] - }, - { - "name": "TestIssue23887", - "cases": [ - "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t" - ] - }, - { - "name": "TestMergeContinuousSelections", - "cases": [ - "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 ;" - ] - }, - { - "name": "TestPushDownGroupConcatToTiFlash", - "cases": [ - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts", - "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", - "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", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts group by col_2", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts group by col_2", - "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", - "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", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_1, id order by col_0) from ts group by col_2", - "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", - "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", - "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", - "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", - "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", - "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", - "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", - "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", - "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", - "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", - "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", - "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", - "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", - "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", - "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", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'01') from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,1) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,10) from ts group by '010'", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts group by '011'", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts group by 'GG'", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG','GG') from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'Gg','GG') from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG-10','GG') from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct '1200-01-01 00:00:00.023',1200) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0) from ts group by id", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0,id) from ts group by id", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts group by col_1", - "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", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by col_0<=>null) from ts" - ] - }, - { - "name": "TestIssue32632", - "cases": [ - "explain format = 'brief' select sum(ps_supplycost) from partsupp, supplier where ps_suppkey = s_suppkey;" - ] - }, - { - "name": "TestTiFlashPartitionTableScan", - "cases": [ - "explain format = 'brief' select * from rp_t where a = 1 or a = 20", - "explain format = 'brief' select * from hp_t where a = 1 or a = 20", - "explain format = 'brief' select count(*) from rp_t where a = 1 or a = 20", - "explain format = 'brief' select count(*) from hp_t where a = 1 or a = 20" - ] - }, - { - "name": "TestTiFlashFineGrainedShuffle", - "cases": [ - // 1. Can use fine grained shuffle. - "explain format = 'brief' select row_number() over w1 from t1 window w1 as (partition by c1 order by c1);", - // Test two window function. - "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);", - // Limit + Order. - "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;", - // // No partition by key in w2, so disabled. But w1 is still enabled. BUG: https://github.com/pingcap/tidb/pull/35256#discussion_r913324160 - // "explain format = 'brief' select row_number() over w1, row_number() over w2 from t1 window w1 as (partition by c1 order by c1), w2 as (order by c1);", - // GroupBy key and window function partition key are not same. - "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);", - "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);", - // Join, same as GroupBy. - "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);", - // Selection. - "explain format = 'brief' select row_number() over w1 from t1 where c1 < 100 window w1 as (partition by c1 order by c1);", - // 2. Cannot use fine grained shuffle. - // No window function, so disabled. - "explain format = 'brief' select * from t1;", - // No partition key in window function, so disabled. - "explain format = 'brief' select row_number() over w1 from t1 window w1 as (order by c1);", - // GroupBy key is same with window function partition key, so they are in one fragment. - // But fine grained shuffle doesn't support group by for now. - "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);" - ] - }, - { - "name": "TestIndexJoinRangeFallback", - "cases": [ - "set @@tidb_opt_range_max_size = 0", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "set @@tidb_opt_range_max_size = 2900", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "set @@tidb_opt_range_max_size = 2300", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "set @@tidb_opt_range_max_size = 700", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "set @@tidb_opt_range_max_size = 0", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", - "set @@tidb_opt_range_max_size = 300", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", - "set @@tidb_opt_range_max_size = 0", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10", - "set @@tidb_opt_range_max_size = 300", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10" - ] - }, - { - "name": "TestNullConditionForPrefixIndex", - "cases": [ - "select count(1) from t1 where c1 = '0xfff' and c2 is not null", - "select count(1) from t1 where c1 = '0xfff' and c2 is null", - "select count(1) from t1 where c1 >= '0xfff' and c2 is not null", - "select count(1) from t1 where c1 >= '0xfff' and c2 is null", - "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is not null", - "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is null", - "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is not null", - "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is null", - "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is not null", - "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is null", - "select count(1) from t2 use index(idx) where b is not null", - "select count(1) from t2 use index(idx) where b is null", - "select b from t2 use index(idx) where b is not null", - "select b from t2 use index(idx) where b is null", - "select b from t3 where a = 1 and b is not null", - "select b from t3 where a = 1 and b is null" - ] - }, - { - "name": "TestDowncastPointGetOrRangeScan", - "cases": [ - "select * from v1 where a = 1; -- the condition should be downcast through both side and go get point", - "select * from v1 where a = '1test'; -- the condition should be downcast through both side and go get point too", - "select * from v1 where a > 1; -- the condition should be downcast through both side and go range scan", - "select * from v2 where a = 'test';", - "select * from v2 where a = 1;", - "select * from v2 where a > 'test';", - "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 too", - "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" - ] - } -] From e8f0ab2a68a83316cc43fd5044bed0a6319a599b Mon Sep 17 00:00:00 2001 From: AilinKid <314806019@qq.com> Date: Tue, 31 Oct 2023 14:28:05 +0800 Subject: [PATCH 5/5] . Signed-off-by: AilinKid <314806019@qq.com> --- planner/core/integration_test.go | 49 +++++++ planner/core/stats.go | 4 - .../core/testdata/integration_suite_in.json | 15 ++ .../core/testdata/integration_suite_out.json | 133 ++++++++++++++++++ 4 files changed, 197 insertions(+), 4 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 83e464f31ec3c..e6e0a13cf450b 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -8434,3 +8434,52 @@ func TestIssue43116(t *testing.T) { " └─TableFullScan 10000.00 cop[tikv] table:a keep order:false, stats:pseudo")) tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 Memory capacity of 111 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen")) } + +func TestDowncastPointGetOrRangeScan(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1 (a bigint key)") + tk.MustExec("create table t2 (a int key)") + tk.MustExec("create definer=`root`@`127.0.0.1` view v1 as (select a from t1) union (select a from t2)") + // select * from v where a = 1 will lead a condition: EQ(cast(t2.a as bigint), 1), + // we should downcast it, utilizing t2.a =1 to walking through the pk point-get. Because cast doesn't contain any precision loss. + + tk.MustExec("create table t3 (a varchar(100) key)") + tk.MustExec("create table t4 (a varchar(10) key)") + tk.MustExec("create definer=`root`@`127.0.0.1` view v2 as (select a from t3) union (select a from t4)") + // select * from v2 where a = 'test' will lead a condition: EQ(cast(t2.a as varchar(100) same collation), 1), + // we should downcast it, utilizing t2.a = 'test' to walking through the pk point-get. Because cast doesn't contain any precision loss. + + tk.MustExec("create table t5 (a char(100) key)") + tk.MustExec("create table t6 (a char(10) key)") + tk.MustExec("create definer=`root`@`127.0.0.1` view v3 as (select a from t5) union (select a from t6)") + // select * from v3 where a = 'test' will lead a condition: EQ(cast(t2.a as char(100) same collation), 1), + // for char type, it depends, with binary collate, the appended '0' after cast column a from char(10) to char(100) will make some difference + // on comparison on where a = 'test' before and after the UNION operator; so we didn't allow this kind of type downcast currently (precision diff). + + tk.MustExec("create table t7 (a varchar(100) key)") + tk.MustExec("create table t8 (a int key)") + tk.MustExec("create definer=`root`@`127.0.0.1` view v4 as (select a from t7) union (select a from t8)") + // since UNION OP will unify the a(int) and a(varchar100) as varchar(100) + // select * from v4 where a = "test" will lead a condition: EQ(cast(t2.a as varchar(100)), "test"), and since + // cast int to varchar(100) may have some precision loss, we couldn't utilize a="test" to get the range directly. + + var input []string + var output []struct { + SQL string + Plan []string + Result []string + } + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + tt).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + }) + tk.MustQuery("explain format='brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Result...)) + } +} diff --git a/planner/core/stats.go b/planner/core/stats.go index 19ca0f6a4b45a..82291661e456f 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -422,12 +422,8 @@ func (ds *DataSource) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, // 1: PushDownNot here can convert query 'not (a != 1)' to 'a = 1'. // 2: EliminateNoPrecisionCast here can convert query 'cast(c as bigint) = 1' to 'c = 1' to leverage access range. for i, expr := range ds.pushedDownConds { -<<<<<<< HEAD - ds.pushedDownConds[i] = expression.PushDownNot(ds.ctx, expr) -======= ds.pushedDownConds[i] = expression.PushDownNot(ds.SCtx(), expr) ds.pushedDownConds[i] = expression.EliminateNoPrecisionLossCast(ds.SCtx(), ds.pushedDownConds[i]) ->>>>>>> 28a9c7f0fb7 (planner: fix cast(col) = range couldn't build range when cast function doesn't contain any precision loss in some cases (#46303)) } for _, path := range ds.possibleAccessPaths { if path.IsTablePath() { diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index a8c3d751691f8..e3137d963e8c2 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -1235,5 +1235,20 @@ "select b from t3 where a = 1 and b is not null", "select b from t3 where a = 1 and b is null" ] + }, + { + "name": "TestDowncastPointGetOrRangeScan", + "cases": [ + "select * from v1 where a = 1; -- the condition should be downcast through both side and go get point", + "select * from v1 where a = '1test'; -- the condition should be downcast through both side and go get point too", + "select * from v1 where a > 1; -- the condition should be downcast through both side and go range scan", + "select * from v2 where a = 'test';", + "select * from v2 where a = 1;", + "select * from v2 where a > 'test';", + "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 too", + "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" + ] } ] diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 3a174a44e9fd3..bae5b56f76ce3 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -10179,5 +10179,138 @@ "Result": null } ] + }, + { + "Name": "TestDowncastPointGetOrRangeScan", + "Cases": [ + { + "SQL": "select * from v1 where a = 1; -- the condition should be downcast through both side and go get point", + "Plan": [ + "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(test.t2.a, bigint(20) BINARY)->Column#3", + " └─Point_Get 1.00 root table:t2 handle:1" + ], + "Result": null + }, + { + "SQL": "select * from v1 where a = '1test'; -- the condition should be downcast through both side and go get point too", + "Plan": [ + "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(test.t2.a, bigint(20) BINARY)->Column#3", + " └─Point_Get 1.00 root table:t2 handle:1" + ], + "Result": null + }, + { + "SQL": "select * from v1 where a > 1; -- the condition should be downcast through both side and go range scan", + "Plan": [ + "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(test.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" + ], + "Result": null + }, + { + "SQL": "select * from v2 where a = 'test';", + "Plan": [ + "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(test.t4.a, varchar(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3", + " └─Point_Get 1.00 root table:t4, clustered index:PRIMARY(a) " + ], + "Result": null + }, + { + "SQL": "select * from v2 where a = 1;", + "Plan": [ + "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(test.t3.a, double BINARY), 1)", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─Projection 8000.00 root cast(test.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(test.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" + ], + "Result": null + }, + { + "SQL": "select * from v2 where a > 'test';", + "Plan": [ + "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(test.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" + ], + "Result": null + }, + { + "SQL": "select * from v3 where a = 'test' -- the condition shouldn't be downcast through both side and go get point", + "Plan": [ + "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(test.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(test.t6.a, char(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), \"test\")", + " └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo" + ], + "Result": null + }, + { + "SQL": "select * from v3 where a > 'test' -- the condition shouldn't be downcast through both side and go get point too", + "Plan": [ + "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(test.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(test.t6.a, char(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), \"test\")", + " └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo" + ], + "Result": null + }, + { + "SQL": "select * from v4 where a = 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range", + "Plan": [ + "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) ", + " └─Projection 8000.00 root cast(test.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3", + " └─TableReader 8000.00 root data:Selection", + " └─Selection 8000.00 cop[tikv] eq(cast(test.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), \"test\")", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Result": null + }, + { + "SQL": "select * from v4 where a > 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range", + "Plan": [ + "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", + " └─Projection 8000.00 root cast(test.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3", + " └─TableReader 8000.00 root data:Selection", + " └─Selection 8000.00 cop[tikv] gt(cast(test.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), \"test\")", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Result": null + } + ] } ]