From fda62be3a315ff62385ad4ce0aaa4dc3bb6df3aa Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Tue, 14 Feb 2023 23:58:06 +0800 Subject: [PATCH 01/12] *: improve planner/core to make test faster Signed-off-by: Weizhen Wang --- planner/core/BUILD.bazel | 7 +- planner/core/binary_plan_test.go | 113 ------- planner/core/casetest/BUILD.bazel | 54 ++++ planner/core/casetest/binary_plan_test.go | 141 ++++++++ .../core/casetest/expression_rewriter_test.go | 81 +++++ planner/core/casetest/main_test.go | 146 +++++++++ planner/core/casetest/plan_test.go | 233 +++++++++++++ planner/core/casetest/point_get_plan_test.go | 95 ++++++ .../predicate_simplification_test.go | 4 +- .../rule_derive_topn_from_window_test.go | 7 +- .../rule_inject_extra_projection_test.go | 5 +- .../core/casetest/rule_join_reorder_test.go | 305 ++++++++++++++++++ .../core/casetest/rule_result_reorder_test.go | 129 ++++++++ planner/core/{ => casetest}/stats_test.go | 8 +- .../analyzeSuiteTestIndexEqualUnknownT.json | 0 ...analyzeSuiteTestLimitIndexEstimationT.json | 0 ...lyzeSuiteTestLowSelIndexGreedySearchT.json | 0 .../testdata/analyze_suite_in.json | 0 .../testdata/analyze_suite_out.json | 0 .../testdata/analyzesSuiteTestIndexReadT.json | 0 .../testdata/binary_plan_suite_in.json | 0 .../testdata/binary_plan_suite_out.json | 0 .../testdata/derive_topn_from_window_in.json | 0 .../testdata/derive_topn_from_window_out.json | 0 .../testdata/enforce_mpp_suite_in.json | 0 .../testdata/enforce_mpp_suite_out.json | 0 .../expression_rewriter_suite_in.json | 0 .../expression_rewriter_suite_out.json | 0 .../testdata/flat_plan_suite_in.json | 0 .../testdata/flat_plan_suite_out.json | 0 .../testdata/index_merge_suite_in.json | 0 .../testdata/index_merge_suite_out.json | 0 .../integration_partition_suite_in.json | 0 .../integration_partition_suite_out.json | 0 .../integration_serial_suite_out.json | 0 .../testdata/integration_suite_in.json | 0 .../testdata/integration_suite_out.json | 0 .../testdata/join_reorder_suite_in.json | 0 .../testdata/join_reorder_suite_out.json | 0 .../testdata/json_plan_suite_in.json | 0 .../testdata/json_plan_suite_out.json | 0 .../ordered_result_mode_suite_in.json | 0 .../ordered_result_mode_suite_out.json | 0 .../testdata/partition_pruner_in.json | 0 .../testdata/partition_pruner_out.json | 0 .../testdata/plan_normalized_suite_in.json | 0 .../testdata/plan_normalized_suite_out.json | 0 .../testdata/plan_suite_in.json | 0 .../testdata/plan_suite_out.json | 0 .../testdata/plan_suite_unexported_in.json | 0 .../testdata/plan_suite_unexported_out.json | 0 .../testdata/point_get_plan_in.json | 0 .../testdata/point_get_plan_out.json | 0 .../testdata/stats_suite_in.json | 0 .../testdata/stats_suite_out.json | 0 .../testdata/window_push_down_suite_in.json | 0 .../testdata/window_push_down_suite_out.json | 0 .../{ => casetest}/window_push_down_test.go | 37 +-- planner/core/expression_rewriter_test.go | 60 ---- planner/core/internal/BUILD.bazel | 15 + planner/core/internal/testkit.go | 50 +++ planner/core/main_test.go | 108 +------ planner/core/plan_test.go | 208 ------------ planner/core/point_get_plan_test.go | 72 ----- planner/core/rule_inject_extra_projection.go | 15 +- planner/core/rule_join_reorder_test.go | 284 ---------------- planner/core/rule_result_reorder_test.go | 107 ------ 67 files changed, 1276 insertions(+), 1008 deletions(-) create mode 100644 planner/core/casetest/BUILD.bazel create mode 100644 planner/core/casetest/binary_plan_test.go create mode 100644 planner/core/casetest/expression_rewriter_test.go create mode 100644 planner/core/casetest/main_test.go create mode 100644 planner/core/casetest/plan_test.go create mode 100644 planner/core/casetest/point_get_plan_test.go rename planner/core/{ => casetest}/predicate_simplification_test.go (97%) rename planner/core/{ => casetest}/rule_derive_topn_from_window_test.go (93%) rename planner/core/{ => casetest}/rule_inject_extra_projection_test.go (94%) create mode 100644 planner/core/casetest/rule_join_reorder_test.go create mode 100644 planner/core/casetest/rule_result_reorder_test.go rename planner/core/{ => casetest}/stats_test.go (96%) rename planner/core/{ => casetest}/testdata/analyzeSuiteTestIndexEqualUnknownT.json (100%) rename planner/core/{ => casetest}/testdata/analyzeSuiteTestLimitIndexEstimationT.json (100%) rename planner/core/{ => casetest}/testdata/analyzeSuiteTestLowSelIndexGreedySearchT.json (100%) rename planner/core/{ => casetest}/testdata/analyze_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/analyze_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/analyzesSuiteTestIndexReadT.json (100%) rename planner/core/{ => casetest}/testdata/binary_plan_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/binary_plan_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/derive_topn_from_window_in.json (100%) rename planner/core/{ => casetest}/testdata/derive_topn_from_window_out.json (100%) rename planner/core/{ => casetest}/testdata/enforce_mpp_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/enforce_mpp_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/expression_rewriter_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/expression_rewriter_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/flat_plan_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/flat_plan_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/index_merge_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/index_merge_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/integration_partition_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/integration_partition_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/integration_serial_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/integration_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/integration_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/join_reorder_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/join_reorder_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/json_plan_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/json_plan_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/ordered_result_mode_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/ordered_result_mode_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/partition_pruner_in.json (100%) rename planner/core/{ => casetest}/testdata/partition_pruner_out.json (100%) rename planner/core/{ => casetest}/testdata/plan_normalized_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/plan_normalized_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/plan_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/plan_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/plan_suite_unexported_in.json (100%) rename planner/core/{ => casetest}/testdata/plan_suite_unexported_out.json (100%) rename planner/core/{ => casetest}/testdata/point_get_plan_in.json (100%) rename planner/core/{ => casetest}/testdata/point_get_plan_out.json (100%) rename planner/core/{ => casetest}/testdata/stats_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/stats_suite_out.json (100%) rename planner/core/{ => casetest}/testdata/window_push_down_suite_in.json (100%) rename planner/core/{ => casetest}/testdata/window_push_down_suite_out.json (100%) rename planner/core/{ => casetest}/window_push_down_test.go (80%) create mode 100644 planner/core/internal/BUILD.bazel create mode 100644 planner/core/internal/testkit.go diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index e6a8af98ae93a..fdb55dcac7522 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -97,6 +97,7 @@ go_library( "//parser/opcode", "//parser/terror", "//parser/types", + "//planner/core/internal", "//planner/funcdep", "//planner/property", "//planner/util", @@ -200,17 +201,12 @@ go_test( "plan_to_pb_test.go", "planbuilder_test.go", "point_get_plan_test.go", - "predicate_simplification_test.go", "prepare_test.go", "preprocess_test.go", - "rule_derive_topn_from_window_test.go", - "rule_inject_extra_projection_test.go", "rule_join_reorder_dp_test.go", "rule_join_reorder_test.go", "rule_result_reorder_test.go", - "stats_test.go", "stringer_test.go", - "window_push_down_test.go", ], data = glob(["testdata/**"]), embed = [":core"], @@ -252,7 +248,6 @@ go_test( "//testkit/ddlhelper", "//testkit/external", "//testkit/testdata", - "//testkit/testmain", "//testkit/testsetup", "//testkit/testutil", "//types", diff --git a/planner/core/binary_plan_test.go b/planner/core/binary_plan_test.go index 46815d7ab620b..e530e20bf8262 100644 --- a/planner/core/binary_plan_test.go +++ b/planner/core/binary_plan_test.go @@ -19,14 +19,12 @@ import ( "fmt" "io" "os" - "regexp" "strings" "testing" "github.com/golang/snappy" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/parser/auth" - "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" @@ -36,117 +34,6 @@ import ( "github.com/stretchr/testify/require" ) -func simplifyAndCheckBinaryPlan(t *testing.T, pb *tipb.ExplainData) { - if pb.Main != nil { - simplifyAndCheckBinaryOperator(t, pb.Main, pb.WithRuntimeStats) - } - for _, cte := range pb.Ctes { - if cte != nil { - simplifyAndCheckBinaryOperator(t, cte, pb.WithRuntimeStats) - } - } -} - -func simplifyAndCheckBinaryOperator(t *testing.T, pb *tipb.ExplainOperator, withRuntimeStats bool) { - if withRuntimeStats { - if pb.TaskType == tipb.TaskType_root { - require.NotEmpty(t, pb.RootBasicExecInfo) - } else if pb.TaskType != tipb.TaskType_unknown { - require.NotEmpty(t, pb.CopExecInfo) - } - } - pb.RootBasicExecInfo = "" - pb.RootGroupExecInfo = nil - pb.CopExecInfo = "" - match, err := regexp.MatchString("((Table|Index).*Scan)|CTEFullScan|Point_Get", pb.Name) - if err == nil && match { - require.NotNil(t, pb.AccessObjects) - } - // AccessObject field is an interface and json.Unmarshall can't handle it, so we don't check it against the json output. - pb.AccessObjects = nil - // MemoryBytes and DiskBytes are not stable sometimes. - pb.MemoryBytes = 0 - pb.DiskBytes = 0 - if len(pb.Children) > 0 { - for _, op := range pb.Children { - if op != nil { - simplifyAndCheckBinaryOperator(t, op, withRuntimeStats) - } - } - } -} - -func TestBinaryPlanInExplainAndSlowLog(t *testing.T) { - // Prepare the slow log - originCfg := config.GetGlobalConfig() - newCfg := *originCfg - f, err := os.CreateTemp("", "tidb-slow-*.log") - require.NoError(t, err) - newCfg.Log.SlowQueryFile = f.Name() - config.StoreGlobalConfig(&newCfg) - defer func() { - config.StoreGlobalConfig(originCfg) - require.NoError(t, f.Close()) - require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) - }() - require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - // If we don't set this, it will be false sometimes and the cost in the result will be different. - tk.MustExec("set @@tidb_enable_chunk_rpc=true") - tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) - tk.MustExec("set tidb_slow_log_threshold=0;") - defer func() { - tk.MustExec("set tidb_slow_log_threshold=300;") - }() - - var input []string - var output []struct { - SQL string - BinaryPlan *tipb.ExplainData - } - planSuiteData := core.GetBinaryPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - - for i, test := range input { - comment := fmt.Sprintf("case:%v sql:%s", i, test) - if len(test) < 7 || test[:7] != "explain" { - tk.MustExec(test) - testdata.OnRecord(func() { - output[i].SQL = test - output[i].BinaryPlan = nil - }) - continue - } - result := testdata.ConvertRowsToStrings(tk.MustQuery(test).Rows()) - require.Equal(t, len(result), 1, comment) - s := result[0] - - // assert that the binary plan in the slow log is the same as the result in the EXPLAIN statement - slowLogResult := testdata.ConvertRowsToStrings(tk.MustQuery("select binary_plan from information_schema.slow_query " + - `where query = "` + test + `;" ` + - "order by time desc limit 1").Rows()) - require.Lenf(t, slowLogResult, 1, comment) - require.Equal(t, s, slowLogResult[0], comment) - - b, err := base64.StdEncoding.DecodeString(s) - require.NoError(t, err) - b, err = snappy.Decode(nil, b) - require.NoError(t, err) - binary := &tipb.ExplainData{} - err = binary.Unmarshal(b) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = test - output[i].BinaryPlan = binary - }) - simplifyAndCheckBinaryPlan(t, binary) - require.Equal(t, output[i].BinaryPlan, binary) - } -} - func TestBinaryPlanSwitch(t *testing.T) { originCfg := config.GetGlobalConfig() newCfg := *originCfg diff --git a/planner/core/casetest/BUILD.bazel b/planner/core/casetest/BUILD.bazel new file mode 100644 index 0000000000000..7f9196de4bd1b --- /dev/null +++ b/planner/core/casetest/BUILD.bazel @@ -0,0 +1,54 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "casetest_test", + timeout = "short", + srcs = [ + "binary_plan_test.go", + "expression_rewriter_test.go", + "main_test.go", + "plan_test.go", + "point_get_plan_test.go", + "predicate_simplification_test.go", + "rule_derive_topn_from_window_test.go", + "rule_inject_extra_projection_test.go", + "rule_join_reorder_test.go", + "rule_result_reorder_test.go", + "stats_test.go", + "window_push_down_test.go", + ], + data = glob(["testdata/**"]), + flaky = True, + shard_count = 50, + deps = [ + "//config", + "//domain", + "//expression", + "//expression/aggregation", + "//infoschema", + "//parser", + "//parser/ast", + "//parser/model", + "//parser/mysql", + "//planner", + "//planner/core", + "//planner/core/internal", + "//planner/property", + "//sessionctx/variable", + "//sessiontxn", + "//testkit", + "//testkit/testdata", + "//testkit/testmain", + "//testkit/testsetup", + "//types", + "//util/hint", + "//util/logutil", + "//util/mock", + "//util/plancodec", + "@com_github_golang_snappy//:snappy", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_pingcap_tipb//go-tipb", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/planner/core/casetest/binary_plan_test.go b/planner/core/casetest/binary_plan_test.go new file mode 100644 index 0000000000000..aa3cb8078cc5e --- /dev/null +++ b/planner/core/casetest/binary_plan_test.go @@ -0,0 +1,141 @@ +// 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 ( + "encoding/base64" + "fmt" + "os" + "regexp" + "testing" + + "github.com/golang/snappy" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tipb/go-tipb" + "github.com/stretchr/testify/require" +) + +func simplifyAndCheckBinaryOperator(t *testing.T, pb *tipb.ExplainOperator, withRuntimeStats bool) { + if withRuntimeStats { + if pb.TaskType == tipb.TaskType_root { + require.NotEmpty(t, pb.RootBasicExecInfo) + } else if pb.TaskType != tipb.TaskType_unknown { + require.NotEmpty(t, pb.CopExecInfo) + } + } + pb.RootBasicExecInfo = "" + pb.RootGroupExecInfo = nil + pb.CopExecInfo = "" + match, err := regexp.MatchString("((Table|Index).*Scan)|CTEFullScan|Point_Get", pb.Name) + if err == nil && match { + require.NotNil(t, pb.AccessObjects) + } + // AccessObject field is an interface and json.Unmarshall can't handle it, so we don't check it against the json output. + pb.AccessObjects = nil + // MemoryBytes and DiskBytes are not stable sometimes. + pb.MemoryBytes = 0 + pb.DiskBytes = 0 + if len(pb.Children) > 0 { + for _, op := range pb.Children { + if op != nil { + simplifyAndCheckBinaryOperator(t, op, withRuntimeStats) + } + } + } +} + +func simplifyAndCheckBinaryPlan(t *testing.T, pb *tipb.ExplainData) { + if pb.Main != nil { + simplifyAndCheckBinaryOperator(t, pb.Main, pb.WithRuntimeStats) + } + for _, cte := range pb.Ctes { + if cte != nil { + simplifyAndCheckBinaryOperator(t, cte, pb.WithRuntimeStats) + } + } +} +func TestBinaryPlanInExplainAndSlowLog(t *testing.T) { + // Prepare the slow log + originCfg := config.GetGlobalConfig() + newCfg := *originCfg + f, err := os.CreateTemp("", "tidb-slow-*.log") + require.NoError(t, err) + newCfg.Log.SlowQueryFile = f.Name() + config.StoreGlobalConfig(&newCfg) + defer func() { + config.StoreGlobalConfig(originCfg) + require.NoError(t, f.Close()) + require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) + }() + require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + // If we don't set this, it will be false sometimes and the cost in the result will be different. + tk.MustExec("set @@tidb_enable_chunk_rpc=true") + tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) + tk.MustExec("set tidb_slow_log_threshold=0;") + defer func() { + tk.MustExec("set tidb_slow_log_threshold=300;") + }() + + var input []string + var output []struct { + SQL string + BinaryPlan *tipb.ExplainData + } + planSuiteData := GetBinaryPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + for i, test := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, test) + if len(test) < 7 || test[:7] != "explain" { + tk.MustExec(test) + testdata.OnRecord(func() { + output[i].SQL = test + output[i].BinaryPlan = nil + }) + continue + } + result := testdata.ConvertRowsToStrings(tk.MustQuery(test).Rows()) + require.Equal(t, len(result), 1, comment) + s := result[0] + + // assert that the binary plan in the slow log is the same as the result in the EXPLAIN statement + slowLogResult := testdata.ConvertRowsToStrings(tk.MustQuery("select binary_plan from information_schema.slow_query " + + `where query = "` + test + `;" ` + + "order by time desc limit 1").Rows()) + require.Lenf(t, slowLogResult, 1, comment) + require.Equal(t, s, slowLogResult[0], comment) + + b, err := base64.StdEncoding.DecodeString(s) + require.NoError(t, err) + b, err = snappy.Decode(nil, b) + require.NoError(t, err) + binary := &tipb.ExplainData{} + err = binary.Unmarshal(b) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = test + output[i].BinaryPlan = binary + }) + simplifyAndCheckBinaryPlan(t, binary) + require.Equal(t, output[i].BinaryPlan, binary) + } +} diff --git a/planner/core/casetest/expression_rewriter_test.go b/planner/core/casetest/expression_rewriter_test.go new file mode 100644 index 0000000000000..67685614a3800 --- /dev/null +++ b/planner/core/casetest/expression_rewriter_test.go @@ -0,0 +1,81 @@ +// 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 ( + "testing" + + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" +) + +func TestMultiColInExpression(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)") + tk.MustExec("insert into t1 values(1,1),(2,null),(null,3),(4,4)") + tk.MustExec("analyze table t1") + tk.MustExec("create table t2(a int, b int)") + tk.MustExec("insert into t2 values(1,1),(2,null),(null,3),(5,4)") + tk.MustExec("analyze table t2") + var input []string + var output []struct { + SQL string + Plan []string + Res []string + } + + // Default RPC encoding may cause statistics explain result differ and then the test unstable. + tk.MustExec("set @@tidb_enable_chunk_rpc = on") + + expressionRewriterSuiteData := GetExpressionRewriterSuiteData() + expressionRewriterSuiteData.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...)) + } +} + +func TestBitFuncsReturnType(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("set tidb_cost_model_version=2") + tk.MustExec("create table t (a timestamp, b varbinary(32))") + tk.MustExec("insert into t values ('2006-08-27 21:57:57', 0x373037343631313230)") + tk.MustExec("analyze table t") + var input []string + var output []struct { + Plan []string + } + + expressionRewriterSuiteData := GetExpressionRewriterSuiteData() + expressionRewriterSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/planner/core/casetest/main_test.go b/planner/core/casetest/main_test.go new file mode 100644 index 0000000000000..610faceee9bd2 --- /dev/null +++ b/planner/core/casetest/main_test.go @@ -0,0 +1,146 @@ +// Copyright 2021 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 ( + "flag" + "testing" + + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/testkit/testmain" + "github.com/pingcap/tidb/testkit/testsetup" + "go.uber.org/goleak" +) + +var testDataMap = make(testdata.BookKeeper) +var indexMergeSuiteData testdata.TestData +var planSuiteUnexportedData testdata.TestData + +func TestMain(m *testing.M) { + testsetup.SetupForCommonTest() + + flag.Parse() + + testDataMap.LoadTestSuiteData("testdata", "integration_partition_suite") + testDataMap.LoadTestSuiteData("testdata", "index_merge_suite") + testDataMap.LoadTestSuiteData("testdata", "plan_normalized_suite") + testDataMap.LoadTestSuiteData("testdata", "stats_suite") + testDataMap.LoadTestSuiteData("testdata", "ordered_result_mode_suite") + testDataMap.LoadTestSuiteData("testdata", "point_get_plan") + testDataMap.LoadTestSuiteData("testdata", "enforce_mpp_suite") + testDataMap.LoadTestSuiteData("testdata", "expression_rewriter_suite") + testDataMap.LoadTestSuiteData("testdata", "partition_pruner") + testDataMap.LoadTestSuiteData("testdata", "plan_suite") + testDataMap.LoadTestSuiteData("testdata", "integration_suite") + testDataMap.LoadTestSuiteData("testdata", "analyze_suite") + testDataMap.LoadTestSuiteData("testdata", "window_push_down_suite") + testDataMap.LoadTestSuiteData("testdata", "plan_suite_unexported") + testDataMap.LoadTestSuiteData("testdata", "join_reorder_suite") + testDataMap.LoadTestSuiteData("testdata", "flat_plan_suite") + testDataMap.LoadTestSuiteData("testdata", "binary_plan_suite") + testDataMap.LoadTestSuiteData("testdata", "json_plan_suite") + testDataMap.LoadTestSuiteData("testdata", "derive_topn_from_window") + + indexMergeSuiteData = testDataMap["index_merge_suite"] + planSuiteUnexportedData = testDataMap["plan_suite_unexported"] + + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), + goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/txnkv/transaction.keepAlive"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + + callback := func(i int) int { + testDataMap.GenerateOutputIfNeeded() + return i + } + + goleak.VerifyTestMain(testmain.WrapTestingM(m, callback), opts...) +} + +func GetIntegrationPartitionSuiteData() testdata.TestData { + return testDataMap["integration_partition_suite"] +} + +func GetPlanNormalizedSuiteData() testdata.TestData { + return testDataMap["plan_normalized_suite"] +} + +func GetStatsSuiteData() testdata.TestData { + return testDataMap["stats_suite"] +} + +func GetOrderedResultModeSuiteData() testdata.TestData { + return testDataMap["ordered_result_mode_suite"] +} + +func GetJoinReorderSuiteData() testdata.TestData { + return testDataMap["join_reorder_suite"] +} + +func GetPointGetPlanData() testdata.TestData { + return testDataMap["point_get_plan"] +} + +func GetEnforceMPPSuiteData() testdata.TestData { + return testDataMap["enforce_mpp_suite"] +} + +func GetExpressionRewriterSuiteData() testdata.TestData { + return testDataMap["expression_rewriter_suite"] +} + +func GetPartitionPrunerData() testdata.TestData { + return testDataMap["partition_pruner"] +} + +func GetPlanSuiteData() testdata.TestData { + return testDataMap["plan_suite"] +} + +func GetIntegrationSuiteData() testdata.TestData { + return testDataMap["integration_suite"] +} + +func GetAnalyzeSuiteData() testdata.TestData { + return testDataMap["analyze_suite"] +} + +func GetWindowPushDownSuiteData() testdata.TestData { + return testDataMap["window_push_down_suite"] +} + +func GetFlatPlanSuiteData() testdata.TestData { + return testDataMap["flat_plan_suite"] +} + +func GetBinaryPlanSuiteData() testdata.TestData { + return testDataMap["binary_plan_suite"] +} + +func GetIndexMergeSuiteData() testdata.TestData { + return testDataMap["index_merge_suite"] +} + +func GetJSONPlanSuiteData() testdata.TestData { + return testDataMap["json_plan_suite"] +} + +func GetDerivedTopNSuiteData() testdata.TestData { + return testDataMap["derive_topn_from_window"] +} diff --git a/planner/core/casetest/plan_test.go b/planner/core/casetest/plan_test.go new file mode 100644 index 0000000000000..36ff13902bace --- /dev/null +++ b/planner/core/casetest/plan_test.go @@ -0,0 +1,233 @@ +// 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 ( + "encoding/json" + "strings" + "testing" + + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/util/plancodec" + "github.com/stretchr/testify/require" +) + +func getPlanRows(planStr string) []string { + planStr = strings.Replace(planStr, "\t", " ", -1) + return strings.Split(planStr, "\n") +} + +func compareStringSlice(t *testing.T, ss1, ss2 []string) { + require.Equal(t, len(ss1), len(ss2)) + for i, s := range ss1 { + require.Equal(t, len(s), len(ss2[i])) + } +} + +func TestPreferRangeScan(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(10) NOT NULL AUTO_INCREMENT,`name` varchar(50) NOT NULL DEFAULT 'tidb',`age` int(11) NOT NULL,`addr` varchar(50) DEFAULT 'The ocean of stars',PRIMARY KEY (`id`),KEY `idx_age` (`age`))") + tk.MustExec("insert into test(age) values(5);") + tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") + tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") + tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") + tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") + tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") + tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") + tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") + tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") + tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") + tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") + tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") + tk.MustExec("analyze table test;") + + // 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 + } + planNormalizedSuiteData := GetPlanNormalizedSuiteData() + planNormalizedSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + if i == 0 { + tk.MustExec("set session tidb_opt_prefer_range_scan=0") + } else if i == 1 { + tk.MustExec("set session tidb_opt_prefer_range_scan=1") + } + tk.Session().GetSessionVars().PlanID = 0 + tk.MustExec(tt) + info := tk.Session().ShowProcess() + require.NotNil(t, info) + p, ok := info.Plan.(core.Plan) + require.True(t, ok) + normalized, digest := core.NormalizePlan(p) + + // test the new normalization code + flat := core.FlattenPhysicalPlan(p, false) + newNormalized, newDigest := core.NormalizeFlatPlan(flat) + require.Equal(t, normalized, newNormalized) + require.Equal(t, digest, newDigest) + + normalizedPlan, err := plancodec.DecodeNormalizedPlan(normalized) + normalizedPlanRows := getPlanRows(normalizedPlan) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = normalizedPlanRows + }) + compareStringSlice(t, normalizedPlanRows, output[i].Plan) + } +} + +func TestNormalizedPlan(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_partition_prune_mode='static';") + tk.MustExec("drop table if exists t1,t2,t3,t4") + tk.MustExec("create table t1 (a int key,b int,c int, index (b));") + tk.MustExec("create table t2 (a int key,b int,c int, index (b));") + tk.MustExec("create table t3 (a int key,b int) partition by hash(a) partitions 2;") + tk.MustExec("create table t4 (a int, b int, index(a)) partition by range(a) (partition p0 values less than (10),partition p1 values less than MAXVALUE);") + tk.MustExec("set @@global.tidb_enable_foreign_key=1") + tk.MustExec("set @@foreign_key_checks=1") + tk.MustExec("create table t5 (id int key, id2 int, id3 int, unique index idx2(id2), index idx3(id3));") + tk.MustExec("create table t6 (id int, id2 int, id3 int, index idx_id(id), index idx_id2(id2), " + + "foreign key fk_1 (id) references t5(id) ON UPDATE CASCADE ON DELETE CASCADE, " + + "foreign key fk_2 (id2) references t5(id2) ON UPDATE CASCADE, " + + "foreign key fk_3 (id3) references t5(id3) ON DELETE CASCADE);") + tk.MustExec("insert into t5 values (1,1,1), (2,2,2)") + var input []string + var output []struct { + SQL string + Plan []string + } + planNormalizedSuiteData := GetPlanNormalizedSuiteData() + planNormalizedSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + tk.Session().GetSessionVars().PlanID = 0 + tk.MustExec(tt) + info := tk.Session().ShowProcess() + require.NotNil(t, info) + p, ok := info.Plan.(core.Plan) + require.True(t, ok) + normalized, digest := core.NormalizePlan(p) + + // test the new normalization code + flat := core.FlattenPhysicalPlan(p, false) + newNormalized, newDigest := core.NormalizeFlatPlan(flat) + require.Equal(t, normalized, newNormalized) + require.Equal(t, digest, newDigest) + // Test for GenHintsFromFlatPlan won't panic. + core.GenHintsFromFlatPlan(flat) + + normalizedPlan, err := plancodec.DecodeNormalizedPlan(normalized) + normalizedPlanRows := getPlanRows(normalizedPlan) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = normalizedPlanRows + }) + compareStringSlice(t, normalizedPlanRows, output[i].Plan) + } +} + +func TestNormalizedPlanForDiffStore(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1 (a int, b int, c int, primary key(a))") + tk.MustExec("insert into t1 values(1,1,1), (2,2,2), (3,3,3)") + tbl, 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. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + var input []string + var output []struct { + Digest string + Plan []string + } + planNormalizedSuiteData := GetPlanNormalizedSuiteData() + planNormalizedSuiteData.LoadTestCases(t, &input, &output) + lastDigest := "" + for i, tt := range input { + tk.Session().GetSessionVars().PlanID = 0 + tk.MustExec(tt) + info := tk.Session().ShowProcess() + require.NotNil(t, info) + ep, ok := info.Plan.(*core.Explain) + require.True(t, ok) + normalized, digest := core.NormalizePlan(ep.TargetPlan) + + // test the new normalization code + flat := core.FlattenPhysicalPlan(ep.TargetPlan, false) + newNormalized, newPlanDigest := core.NormalizeFlatPlan(flat) + require.Equal(t, digest, newPlanDigest) + require.Equal(t, normalized, newNormalized) + + normalizedPlan, err := plancodec.DecodeNormalizedPlan(normalized) + normalizedPlanRows := getPlanRows(normalizedPlan) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].Digest = digest.String() + output[i].Plan = normalizedPlanRows + }) + compareStringSlice(t, normalizedPlanRows, output[i].Plan) + require.NotEqual(t, digest.String(), lastDigest) + lastDigest = digest.String() + } +} + +func TestJSONPlanInExplain(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(id int, key(id))") + tk.MustExec("create table t2(id int, key(id))") + + var input []string + var output []struct { + SQL string + JSONPlan []*core.ExplainInfoForEncode + } + planSuiteData := GetJSONPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + for i, test := range input { + resJSON := tk.MustQuery(test).Rows() + var res []*core.ExplainInfoForEncode + require.NoError(t, json.Unmarshal([]byte(resJSON[0][0].(string)), &res)) + for j, expect := range output[i].JSONPlan { + require.Equal(t, expect.ID, res[j].ID) + require.Equal(t, expect.EstRows, res[j].EstRows) + require.Equal(t, expect.ActRows, res[j].ActRows) + require.Equal(t, expect.TaskType, res[j].TaskType) + require.Equal(t, expect.AccessObject, res[j].AccessObject) + require.Equal(t, expect.OperatorInfo, res[j].OperatorInfo) + } + } +} diff --git a/planner/core/casetest/point_get_plan_test.go b/planner/core/casetest/point_get_plan_test.go new file mode 100644 index 0000000000000..ba1827b282039 --- /dev/null +++ b/planner/core/casetest/point_get_plan_test.go @@ -0,0 +1,95 @@ +// 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 ( + "testing" + + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/stretchr/testify/require" +) + +func TestCBOPointGet(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.MustExec("create table t (a varchar(20), b int, c int, d int, primary key(a), unique key(b, c))") + tk.MustExec("insert into t values('1',4,4,1), ('2',3,3,2), ('3',2,2,3), ('4',1,1,4)") + + var input []string + var output []struct { + SQL string + Plan []string + Res []string + } + pointGetPlanData := GetPointGetPlanData() + pointGetPlanData.LoadTestCases(t, &input, &output) + require.Equal(t, len(input), len(output)) + for i, sql := range input { + plan := tk.MustQuery("explain format = 'brief' " + sql) + res := tk.MustQuery(sql) + testdata.OnRecord(func() { + output[i].SQL = sql + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) + output[i].Res = testdata.ConvertRowsToStrings(res.Rows()) + }) + plan.Check(testkit.Rows(output[i].Plan...)) + res.Check(testkit.Rows(output[i].Res...)) + } +} + +func TestCBOShouldNotUsePointGet(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("drop tables if exists t1, t2, t3, t4, t5") + tk.MustExec("create table t1(id varchar(20) primary key)") + tk.MustExec("create table t2(id varchar(20), unique(id))") + tk.MustExec("create table t3(id varchar(20), d varchar(20), unique(id, d))") + tk.MustExec("create table t4(id int, d varchar(20), c varchar(20), unique(id, d))") + tk.MustExec("create table t5(id bit(64) primary key)") + tk.MustExec("insert into t1 values('asdf'), ('1asdf')") + tk.MustExec("insert into t2 values('asdf'), ('1asdf')") + tk.MustExec("insert into t3 values('asdf', 't'), ('1asdf', 't')") + tk.MustExec("insert into t4 values(1, 'b', 'asdf'), (1, 'c', 'jkl'), (1, 'd', '1jkl')") + tk.MustExec("insert into t5 values(48)") + + var input []string + var output []struct { + SQL string + Plan []string + Res []string + } + + pointGetPlanData := GetPointGetPlanData() + pointGetPlanData.LoadTestCases(t, &input, &output) + require.Equal(t, len(input), len(output)) + for i, sql := range input { + plan := tk.MustQuery("explain format = 'brief' " + sql) + res := tk.MustQuery(sql) + testdata.OnRecord(func() { + output[i].SQL = sql + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) + output[i].Res = testdata.ConvertRowsToStrings(res.Rows()) + }) + plan.Check(testkit.Rows(output[i].Plan...)) + res.Check(testkit.Rows(output[i].Res...)) + } +} diff --git a/planner/core/predicate_simplification_test.go b/planner/core/casetest/predicate_simplification_test.go similarity index 97% rename from planner/core/predicate_simplification_test.go rename to planner/core/casetest/predicate_simplification_test.go index 673c793d41013..d590ea5b617db 100644 --- a/planner/core/predicate_simplification_test.go +++ b/planner/core/casetest/predicate_simplification_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core_test +package casetest import ( "context" @@ -43,7 +43,7 @@ func TestRemoveRedundantPredicates(t *testing.T) { SQL string Best string } - planSuiteData := core.GetPlanSuiteData() + planSuiteData := GetPlanSuiteData() planSuiteData.LoadTestCases(t, &input, &output) p := parser.New() is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) diff --git a/planner/core/rule_derive_topn_from_window_test.go b/planner/core/casetest/rule_derive_topn_from_window_test.go similarity index 93% rename from planner/core/rule_derive_topn_from_window_test.go rename to planner/core/casetest/rule_derive_topn_from_window_test.go index c241523457e42..85d718779cfcb 100644 --- a/planner/core/rule_derive_topn_from_window_test.go +++ b/planner/core/casetest/rule_derive_topn_from_window_test.go @@ -12,12 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core_test +package casetest import ( "testing" - plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" ) @@ -39,7 +38,7 @@ func TestPushDerivedTopnNegative(t *testing.T) { SQL string Plan []string } - suiteData := plannercore.GetDerivedTopNSuiteData() + suiteData := GetDerivedTopNSuiteData() suiteData.LoadTestCases(t, &input, &output) for i, sql := range input { plan := tk.MustQuery("explain format = 'brief' " + sql) @@ -69,7 +68,7 @@ func TestPushDerivedTopnPositive(t *testing.T) { Plan []string Res []string } - suiteData := plannercore.GetDerivedTopNSuiteData() + suiteData := GetDerivedTopNSuiteData() suiteData.LoadTestCases(t, &input, &output) for i, sql := range input { plan := tk.MustQuery("explain format = 'brief' " + sql) diff --git a/planner/core/rule_inject_extra_projection_test.go b/planner/core/casetest/rule_inject_extra_projection_test.go similarity index 94% rename from planner/core/rule_inject_extra_projection_test.go rename to planner/core/casetest/rule_inject_extra_projection_test.go index c1e186a247e13..c121d02346300 100644 --- a/planner/core/rule_inject_extra_projection_test.go +++ b/planner/core/casetest/rule_inject_extra_projection_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core +package casetest import ( "testing" @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/core/internal" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" @@ -55,7 +56,7 @@ func TestWrapCastForAggFuncs(t *testing.T) { orgAggFuncs = append(orgAggFuncs, agg.Clone()) } - wrapCastForAggFuncs(mock.NewContext(), aggFuncs) + internal.WrapCastForAggFuncs(mock.NewContext(), aggFuncs) for i := range aggFuncs { if aggFuncs[i].Mode != aggregation.FinalMode && aggFuncs[i].Mode != aggregation.Partial2Mode { require.Equal(t, aggFuncs[i].Args[0].GetType().GetType(), aggFuncs[i].RetTp.GetType()) diff --git a/planner/core/casetest/rule_join_reorder_test.go b/planner/core/casetest/rule_join_reorder_test.go new file mode 100644 index 0000000000000..2b0529da5c367 --- /dev/null +++ b/planner/core/casetest/rule_join_reorder_test.go @@ -0,0 +1,305 @@ +// 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 ( + "testing" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/stretchr/testify/require" +) + +func runJoinReorderTestData(t *testing.T, tk *testkit.TestKit, name string) { + var input []string + var output []struct { + SQL string + Plan []string + Warning []string + } + joinReorderSuiteData := GetJoinReorderSuiteData() + joinReorderSuiteData.LoadTestCasesByName(name, t, &input, &output) + require.Equal(t, len(input), len(output)) + for i := range input { + testdata.OnRecord(func() { + output[i].SQL = input[i] + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + input[i]).Rows()) + output[i].Warning = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery("explain format = 'brief' " + input[i]).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warning...)) + } +} + +func TestStraightJoinHint(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, t1, t2, t3, t4;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + runJoinReorderTestData(t, tk, "TestStraightJoinHint") +} + +func TestLeadingJoinHint(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, t1, t2, t3, t4, t5, t6, t7, t8;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + tk.MustExec("create table t5(a int, b int, key(a));") + tk.MustExec("create table t6(a int, b int, key(a));") + tk.MustExec("create table t7(a int, b int, key(a));") + tk.MustExec("create table t8(a int, b int, key(a));") + runJoinReorderTestData(t, tk, "TestLeadingJoinHint") + + // test cases for multiple leading hints + tk.MustExec("select /*+ leading(t1) leading(t2) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) +} + +func TestJoinOrderHint(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t, t1, t2, t3, t4, t5, t6, t7, t8;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + tk.MustExec("create table t5(a int, b int, key(a));") + tk.MustExec("create table t6(a int, b int, key(a));") + tk.MustExec("create table t7(a int, b int, key(a));") + tk.MustExec("create table t8(a int, b int, key(a));") + + // test cases for using the leading hint and straight_join hint at the same time + tk.MustExec("select /*+ leading(t1) straight_join() */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use the straight_join hint, when we use the leading hint and straight_join hint at the same time, all leading hints will be invalid")) + + tk.MustExec("select /*+ straight_join() leading(t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use the straight_join hint, when we use the leading hint and straight_join hint at the same time, all leading hints will be invalid")) + + // more join order hints appear in the same time + tk.MustExec("select /*+ leading(t1) leading(t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) + + tk.MustExec("select /*+ leading(t1) leading(t2) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) + + tk.MustExec("select /*+ straight_join() straight_join() */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 STRAIGHT_JOIN() is defined more than once, only the last definition takes effect")) + + // test cases for table name in hint + // the same table appears in the leading hint + tk.MustExec("select /*+ leading(t1, t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t1) in optimizer hint /*+ LEADING(t1, t1) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) + + tk.MustExec("select /*+ leading(t1, t2, t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t1) in optimizer hint /*+ LEADING(t1, t2, t1) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) + + // the wrong table appears in the leading hint + tk.MustExec("select /*+ leading(t) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t) in optimizer hint /*+ LEADING(t) */. Maybe you can use the table alias name")) + + tk.MustExec("select /*+ leading(t1, t2, t) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t) in optimizer hint /*+ LEADING(t1, t2, t) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) + + // table alias in the leading hint + tk.MustExec("select /*+ leading(t) */ * from t1 t join t2 on t.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows()) + + tk.MustExec("select /*+ leading(t1) */ * from t1 t join t2 on t.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t1) in optimizer hint /*+ LEADING(t1) */. Maybe you can use the table alias name")) + + tk.MustExec("select /*+ leading(t2, t) */ * from t1 t join t2 on t.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows()) + + tk.MustExec("select /*+ leading(t2, t1) */ * from t1 t join t2 on t.a=t2.a join t3 on t2.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t1) in optimizer hint /*+ LEADING(t2, t1) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) + + // table name in leading hint cross query block + // Todo: Can not handle this case yet. Because when we extract the join group, it will get the join group {t1, t2, t3}. + // So the table 't4' can not be used. + tk.MustExec("select /*+ leading(t4) */ * from (select t2.b from t1 join t2 on t1.a=t2.a) t4 join t3 on t4.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) + + tk.MustExec("select /*+ leading(t3, t2@sel_2) */ * from (select t2.b from t1 join t2 on t1.a=t2.a) t4 join t3 on t4.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t2) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name")) + + tk.MustExec("select * from (select /*+ leading(t1, t3@sel_1) */ t2.b from t1 join t2 on t1.a=t2.a) t4 join t3 on t4.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t1, t3) */. Maybe you can use the table alias name")) + + tk.MustExec("select /*+ leading(t3) */ * from (select /*+ leading(t1) */ t2.b from t1 join t2 on t1.a=t2.a) t4 join t3 on t4.b=t3.b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) + + runJoinReorderTestData(t, tk, "TestJoinOrderHint") +} + +func TestJoinOrderHint4StaticPartitionTable(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, t1, t2, t3;") + tk.MustExec(`create table t(a int, b int) partition by hash(a) partitions 3`) + tk.MustExec(`create table t1(a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table t2(a int, b int) partition by hash(a) partitions 5`) + tk.MustExec(`create table t3(a int, b int) partition by hash(b) partitions 3`) + tk.MustExec(`create table t4(a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table t5(a int, b int) partition by hash(a) partitions 5`) + tk.MustExec(`create table t6(a int, b int) partition by hash(b) partitions 3`) + + tk.MustExec(`set @@tidb_partition_prune_mode="static"`) + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") + runJoinReorderTestData(t, tk, "TestJoinOrderHint4StaticPartitionTable") +} + +func TestJoinOrderHint4DynamicPartitionTable(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, t1, t2, t3;") + tk.MustExec(`create table t(a int, b int) partition by hash(a) partitions 3`) + tk.MustExec(`create table t1(a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table t2(a int, b int) partition by hash(a) partitions 5`) + tk.MustExec(`create table t3(a int, b int) partition by hash(b) partitions 3`) + tk.MustExec(`create table t4(a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table t5(a int, b int) partition by hash(a) partitions 5`) + tk.MustExec(`create table t6(a int, b int) partition by hash(b) partitions 3`) + + tk.MustExec(`set @@tidb_partition_prune_mode="dynamic"`) + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") + runJoinReorderTestData(t, tk, "TestJoinOrderHint4DynamicPartitionTable") +} + +func TestJoinOrderHint4DifferentJoinType(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, t1, t2, t3, t4, t5, t6, t7, t8;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + tk.MustExec("create table t5(a int, b int, key(a));") + tk.MustExec("create table t6(a int, b int, key(a));") + tk.MustExec("create table t7(a int, b int, key(a));") + tk.MustExec("create table t8(a int, b int, key(a));") + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") + + runJoinReorderTestData(t, tk, "TestJoinOrderHint4DifferentJoinType") +} + +func TestJoinOrderHint4TiFlash(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t, t1, t2, t3;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + tk.MustExec("create table t5(a int, b int, key(a));") + tk.MustExec("create table t6(a int, b int, key(a));") + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") + + // 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 { + tableName := tblInfo.Name.L + if tableName == "t" || tableName == "t1" || tableName == "t2" || tableName == "t3" || tableName == "t4" || tableName == "t5" || tableName == "t6" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + runJoinReorderTestData(t, tk, "TestJoinOrderHint4TiFlash") +} + +func TestJoinOrderHint4Subquery(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, t1, t2, t3, t4, t5, t6, t7, t8;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + tk.MustExec("create table t5(a int, b int, key(a));") + tk.MustExec("create table t6(a int, b int, key(a));") + tk.MustExec("create table t7(a int, b int, key(a));") + tk.MustExec("create table t8(a int, b int, key(a));") + tk.MustExec("insert into t3 values(1, 1), (2, 2), (3, 3);") + tk.MustExec("analyze table t3;") + + runJoinReorderTestData(t, tk, "TestJoinOrderHint4Subquery") +} + +func TestLeadingJoinHint4OuterJoin(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, t1, t2, t3, t4, t5, t6, t7, t8;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + tk.MustExec("create table t5(a int, b int, key(a));") + tk.MustExec("create table t6(a int, b int, key(a));") + tk.MustExec("create table t7(a int, b int, key(a));") + tk.MustExec("create table t8(a int, b int, key(a));") + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") + runJoinReorderTestData(t, tk, "TestLeadingJoinHint4OuterJoin") +} diff --git a/planner/core/casetest/rule_result_reorder_test.go b/planner/core/casetest/rule_result_reorder_test.go new file mode 100644 index 0000000000000..aff7d36333c1a --- /dev/null +++ b/planner/core/casetest/rule_result_reorder_test.go @@ -0,0 +1,129 @@ +// 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" + "testing" + + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/stretchr/testify/require" +) + +func runTestData(t *testing.T, tk *testkit.TestKit, name string) { + var input []string + var output []struct { + Plan []string + } + statsSuiteData := GetOrderedResultModeSuiteData() + statsSuiteData.LoadTestCasesByName(name, t, &input, &output) + require.Equal(t, len(input), len(output)) + for i := range input { + testdata.OnRecord(func() { + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + input[i]).Rows()) + }) + tk.MustQuery("explain " + input[i]).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestOrderedResultMode(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("set tidb_enable_ordered_result_mode=1") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int primary key, b int, c int, d int, key(b))") + runTestData(t, tk, "TestOrderedResultMode") +} + +func TestOrderedResultModeOnDML(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_enable_ordered_result_mode=1") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int primary key, b int, c int, key(b))") + runTestData(t, tk, "TestOrderedResultModeOnDML") +} + +func TestOrderedResultModeOnSubQuery(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_enable_ordered_result_mode=1") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t1 (a int primary key, b int, c int, d int, key(b))") + tk.MustExec("create table t2 (a int primary key, b int, c int, d int, key(b))") + runTestData(t, tk, "TestOrderedResultModeOnSubQuery") +} + +func TestOrderedResultModeOnJoin(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_enable_ordered_result_mode=1") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t1 (a int primary key, b int, c int, d int, key(b))") + tk.MustExec("create table t2 (a int primary key, b int, c int, d int, key(b))") + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") + runTestData(t, tk, "TestOrderedResultModeOnJoin") +} + +func TestOrderedResultModeOnOtherOperators(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_enable_ordered_result_mode=1") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t1 (a int primary key, b int, c int, d int, unique key(b))") + tk.MustExec("create table t2 (a int primary key, b int, c int, d int, unique key(b))") + runTestData(t, tk, "TestOrderedResultModeOnOtherOperators") +} + +func TestOrderedResultModeOnPartitionTable(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(fmt.Sprintf(`set tidb_partition_prune_mode='%v'`, variable.DefTiDBPartitionPruneMode)) + tk.MustExec("set tidb_enable_ordered_result_mode=1") + tk.MustExec("drop table if exists thash") + tk.MustExec("drop table if exists trange") + tk.MustExec("create table thash (a int primary key, b int, c int, d int) partition by hash(a) partitions 4") + tk.MustExec(`create table trange (a int primary key, b int, c int, d int) partition by range(a) ( + partition p0 values less than (100), + partition p1 values less than (200), + partition p2 values less than (300), + partition p3 values less than (400))`) + tk.MustExec(`analyze table thash`) + tk.MustExec(`analyze table trange`) + tk.MustQuery("select @@tidb_partition_prune_mode").Check(testkit.Rows("dynamic")) + runTestData(t, tk, "TestOrderedResultModeOnPartitionTable") +} diff --git a/planner/core/stats_test.go b/planner/core/casetest/stats_test.go similarity index 96% rename from planner/core/stats_test.go rename to planner/core/casetest/stats_test.go index 2948c90abb4df..2e82232b227f4 100644 --- a/planner/core/stats_test.go +++ b/planner/core/casetest/stats_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core_test +package casetest import ( "context" @@ -48,7 +48,7 @@ func TestGroupNDVs(t *testing.T) { AggInput string JoinInput string } - statsSuiteData := core.GetStatsSuiteData() + statsSuiteData := GetStatsSuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i, tt := range input { comment := fmt.Sprintf("case:%v sql: %s", i, tt) @@ -61,7 +61,7 @@ func TestGroupNDVs(t *testing.T) { builder, _ := core.NewPlanBuilder().Init(tk.Session(), ret.InfoSchema, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) require.NoError(t, err, comment) - p, err = core.LogicalOptimize(ctx, builder.GetOptFlag(), p.(core.LogicalPlan)) + p, err = core.LogicalOptimizeTest(ctx, builder.GetOptFlag(), p.(core.LogicalPlan)) require.NoError(t, err, comment) lp := p.(core.LogicalPlan) _, err = core.RecursiveDeriveStats4Test(lp) @@ -140,7 +140,7 @@ func TestNDVGroupCols(t *testing.T) { SQL string Plan []string } - statsSuiteData := core.GetStatsSuiteData() + statsSuiteData := GetStatsSuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i, tt := range input { testdata.OnRecord(func() { diff --git a/planner/core/testdata/analyzeSuiteTestIndexEqualUnknownT.json b/planner/core/casetest/testdata/analyzeSuiteTestIndexEqualUnknownT.json similarity index 100% rename from planner/core/testdata/analyzeSuiteTestIndexEqualUnknownT.json rename to planner/core/casetest/testdata/analyzeSuiteTestIndexEqualUnknownT.json diff --git a/planner/core/testdata/analyzeSuiteTestLimitIndexEstimationT.json b/planner/core/casetest/testdata/analyzeSuiteTestLimitIndexEstimationT.json similarity index 100% rename from planner/core/testdata/analyzeSuiteTestLimitIndexEstimationT.json rename to planner/core/casetest/testdata/analyzeSuiteTestLimitIndexEstimationT.json diff --git a/planner/core/testdata/analyzeSuiteTestLowSelIndexGreedySearchT.json b/planner/core/casetest/testdata/analyzeSuiteTestLowSelIndexGreedySearchT.json similarity index 100% rename from planner/core/testdata/analyzeSuiteTestLowSelIndexGreedySearchT.json rename to planner/core/casetest/testdata/analyzeSuiteTestLowSelIndexGreedySearchT.json diff --git a/planner/core/testdata/analyze_suite_in.json b/planner/core/casetest/testdata/analyze_suite_in.json similarity index 100% rename from planner/core/testdata/analyze_suite_in.json rename to planner/core/casetest/testdata/analyze_suite_in.json diff --git a/planner/core/testdata/analyze_suite_out.json b/planner/core/casetest/testdata/analyze_suite_out.json similarity index 100% rename from planner/core/testdata/analyze_suite_out.json rename to planner/core/casetest/testdata/analyze_suite_out.json diff --git a/planner/core/testdata/analyzesSuiteTestIndexReadT.json b/planner/core/casetest/testdata/analyzesSuiteTestIndexReadT.json similarity index 100% rename from planner/core/testdata/analyzesSuiteTestIndexReadT.json rename to planner/core/casetest/testdata/analyzesSuiteTestIndexReadT.json diff --git a/planner/core/testdata/binary_plan_suite_in.json b/planner/core/casetest/testdata/binary_plan_suite_in.json similarity index 100% rename from planner/core/testdata/binary_plan_suite_in.json rename to planner/core/casetest/testdata/binary_plan_suite_in.json diff --git a/planner/core/testdata/binary_plan_suite_out.json b/planner/core/casetest/testdata/binary_plan_suite_out.json similarity index 100% rename from planner/core/testdata/binary_plan_suite_out.json rename to planner/core/casetest/testdata/binary_plan_suite_out.json diff --git a/planner/core/testdata/derive_topn_from_window_in.json b/planner/core/casetest/testdata/derive_topn_from_window_in.json similarity index 100% rename from planner/core/testdata/derive_topn_from_window_in.json rename to planner/core/casetest/testdata/derive_topn_from_window_in.json diff --git a/planner/core/testdata/derive_topn_from_window_out.json b/planner/core/casetest/testdata/derive_topn_from_window_out.json similarity index 100% rename from planner/core/testdata/derive_topn_from_window_out.json rename to planner/core/casetest/testdata/derive_topn_from_window_out.json diff --git a/planner/core/testdata/enforce_mpp_suite_in.json b/planner/core/casetest/testdata/enforce_mpp_suite_in.json similarity index 100% rename from planner/core/testdata/enforce_mpp_suite_in.json rename to planner/core/casetest/testdata/enforce_mpp_suite_in.json diff --git a/planner/core/testdata/enforce_mpp_suite_out.json b/planner/core/casetest/testdata/enforce_mpp_suite_out.json similarity index 100% rename from planner/core/testdata/enforce_mpp_suite_out.json rename to planner/core/casetest/testdata/enforce_mpp_suite_out.json diff --git a/planner/core/testdata/expression_rewriter_suite_in.json b/planner/core/casetest/testdata/expression_rewriter_suite_in.json similarity index 100% rename from planner/core/testdata/expression_rewriter_suite_in.json rename to planner/core/casetest/testdata/expression_rewriter_suite_in.json diff --git a/planner/core/testdata/expression_rewriter_suite_out.json b/planner/core/casetest/testdata/expression_rewriter_suite_out.json similarity index 100% rename from planner/core/testdata/expression_rewriter_suite_out.json rename to planner/core/casetest/testdata/expression_rewriter_suite_out.json diff --git a/planner/core/testdata/flat_plan_suite_in.json b/planner/core/casetest/testdata/flat_plan_suite_in.json similarity index 100% rename from planner/core/testdata/flat_plan_suite_in.json rename to planner/core/casetest/testdata/flat_plan_suite_in.json diff --git a/planner/core/testdata/flat_plan_suite_out.json b/planner/core/casetest/testdata/flat_plan_suite_out.json similarity index 100% rename from planner/core/testdata/flat_plan_suite_out.json rename to planner/core/casetest/testdata/flat_plan_suite_out.json diff --git a/planner/core/testdata/index_merge_suite_in.json b/planner/core/casetest/testdata/index_merge_suite_in.json similarity index 100% rename from planner/core/testdata/index_merge_suite_in.json rename to planner/core/casetest/testdata/index_merge_suite_in.json diff --git a/planner/core/testdata/index_merge_suite_out.json b/planner/core/casetest/testdata/index_merge_suite_out.json similarity index 100% rename from planner/core/testdata/index_merge_suite_out.json rename to planner/core/casetest/testdata/index_merge_suite_out.json diff --git a/planner/core/testdata/integration_partition_suite_in.json b/planner/core/casetest/testdata/integration_partition_suite_in.json similarity index 100% rename from planner/core/testdata/integration_partition_suite_in.json rename to planner/core/casetest/testdata/integration_partition_suite_in.json diff --git a/planner/core/testdata/integration_partition_suite_out.json b/planner/core/casetest/testdata/integration_partition_suite_out.json similarity index 100% rename from planner/core/testdata/integration_partition_suite_out.json rename to planner/core/casetest/testdata/integration_partition_suite_out.json diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/casetest/testdata/integration_serial_suite_out.json similarity index 100% rename from planner/core/testdata/integration_serial_suite_out.json rename to planner/core/casetest/testdata/integration_serial_suite_out.json diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/casetest/testdata/integration_suite_in.json similarity index 100% rename from planner/core/testdata/integration_suite_in.json rename to planner/core/casetest/testdata/integration_suite_in.json diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/casetest/testdata/integration_suite_out.json similarity index 100% rename from planner/core/testdata/integration_suite_out.json rename to planner/core/casetest/testdata/integration_suite_out.json diff --git a/planner/core/testdata/join_reorder_suite_in.json b/planner/core/casetest/testdata/join_reorder_suite_in.json similarity index 100% rename from planner/core/testdata/join_reorder_suite_in.json rename to planner/core/casetest/testdata/join_reorder_suite_in.json diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/casetest/testdata/join_reorder_suite_out.json similarity index 100% rename from planner/core/testdata/join_reorder_suite_out.json rename to planner/core/casetest/testdata/join_reorder_suite_out.json diff --git a/planner/core/testdata/json_plan_suite_in.json b/planner/core/casetest/testdata/json_plan_suite_in.json similarity index 100% rename from planner/core/testdata/json_plan_suite_in.json rename to planner/core/casetest/testdata/json_plan_suite_in.json diff --git a/planner/core/testdata/json_plan_suite_out.json b/planner/core/casetest/testdata/json_plan_suite_out.json similarity index 100% rename from planner/core/testdata/json_plan_suite_out.json rename to planner/core/casetest/testdata/json_plan_suite_out.json diff --git a/planner/core/testdata/ordered_result_mode_suite_in.json b/planner/core/casetest/testdata/ordered_result_mode_suite_in.json similarity index 100% rename from planner/core/testdata/ordered_result_mode_suite_in.json rename to planner/core/casetest/testdata/ordered_result_mode_suite_in.json diff --git a/planner/core/testdata/ordered_result_mode_suite_out.json b/planner/core/casetest/testdata/ordered_result_mode_suite_out.json similarity index 100% rename from planner/core/testdata/ordered_result_mode_suite_out.json rename to planner/core/casetest/testdata/ordered_result_mode_suite_out.json diff --git a/planner/core/testdata/partition_pruner_in.json b/planner/core/casetest/testdata/partition_pruner_in.json similarity index 100% rename from planner/core/testdata/partition_pruner_in.json rename to planner/core/casetest/testdata/partition_pruner_in.json diff --git a/planner/core/testdata/partition_pruner_out.json b/planner/core/casetest/testdata/partition_pruner_out.json similarity index 100% rename from planner/core/testdata/partition_pruner_out.json rename to planner/core/casetest/testdata/partition_pruner_out.json diff --git a/planner/core/testdata/plan_normalized_suite_in.json b/planner/core/casetest/testdata/plan_normalized_suite_in.json similarity index 100% rename from planner/core/testdata/plan_normalized_suite_in.json rename to planner/core/casetest/testdata/plan_normalized_suite_in.json diff --git a/planner/core/testdata/plan_normalized_suite_out.json b/planner/core/casetest/testdata/plan_normalized_suite_out.json similarity index 100% rename from planner/core/testdata/plan_normalized_suite_out.json rename to planner/core/casetest/testdata/plan_normalized_suite_out.json diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/casetest/testdata/plan_suite_in.json similarity index 100% rename from planner/core/testdata/plan_suite_in.json rename to planner/core/casetest/testdata/plan_suite_in.json diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/casetest/testdata/plan_suite_out.json similarity index 100% rename from planner/core/testdata/plan_suite_out.json rename to planner/core/casetest/testdata/plan_suite_out.json diff --git a/planner/core/testdata/plan_suite_unexported_in.json b/planner/core/casetest/testdata/plan_suite_unexported_in.json similarity index 100% rename from planner/core/testdata/plan_suite_unexported_in.json rename to planner/core/casetest/testdata/plan_suite_unexported_in.json diff --git a/planner/core/testdata/plan_suite_unexported_out.json b/planner/core/casetest/testdata/plan_suite_unexported_out.json similarity index 100% rename from planner/core/testdata/plan_suite_unexported_out.json rename to planner/core/casetest/testdata/plan_suite_unexported_out.json diff --git a/planner/core/testdata/point_get_plan_in.json b/planner/core/casetest/testdata/point_get_plan_in.json similarity index 100% rename from planner/core/testdata/point_get_plan_in.json rename to planner/core/casetest/testdata/point_get_plan_in.json diff --git a/planner/core/testdata/point_get_plan_out.json b/planner/core/casetest/testdata/point_get_plan_out.json similarity index 100% rename from planner/core/testdata/point_get_plan_out.json rename to planner/core/casetest/testdata/point_get_plan_out.json diff --git a/planner/core/testdata/stats_suite_in.json b/planner/core/casetest/testdata/stats_suite_in.json similarity index 100% rename from planner/core/testdata/stats_suite_in.json rename to planner/core/casetest/testdata/stats_suite_in.json diff --git a/planner/core/testdata/stats_suite_out.json b/planner/core/casetest/testdata/stats_suite_out.json similarity index 100% rename from planner/core/testdata/stats_suite_out.json rename to planner/core/casetest/testdata/stats_suite_out.json diff --git a/planner/core/testdata/window_push_down_suite_in.json b/planner/core/casetest/testdata/window_push_down_suite_in.json similarity index 100% rename from planner/core/testdata/window_push_down_suite_in.json rename to planner/core/casetest/testdata/window_push_down_suite_in.json diff --git a/planner/core/testdata/window_push_down_suite_out.json b/planner/core/casetest/testdata/window_push_down_suite_out.json similarity index 100% rename from planner/core/testdata/window_push_down_suite_out.json rename to planner/core/casetest/testdata/window_push_down_suite_out.json diff --git a/planner/core/window_push_down_test.go b/planner/core/casetest/window_push_down_test.go similarity index 80% rename from planner/core/window_push_down_test.go rename to planner/core/casetest/window_push_down_test.go index 0d8538fa19ca0..7e3f08845b6ed 100644 --- a/planner/core/window_push_down_test.go +++ b/planner/core/casetest/window_push_down_test.go @@ -1,4 +1,4 @@ -// Copyright 2019 PingCAP, Inc. +// 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. @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core_test +package casetest import ( "strings" @@ -20,27 +20,12 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/parser/model" - plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/planner/core/internal" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" "github.com/stretchr/testify/require" ) -func SetTiFlashReplica(t *testing.T, dom *domain.Domain, dbName, tableName string) { - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr(dbName)) - require.True(t, exists) - for _, tblInfo := range db.Tables { - if tblInfo.Name.L == tableName { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } -} - type Input []string type Output []struct { SQL string @@ -78,11 +63,11 @@ func TestWindowFunctionDescCanPushDown(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists employee") tk.MustExec("create table employee (empid int, deptid int, salary decimal(10,2))") - SetTiFlashReplica(t, dom, "test", "employee") + internal.SetTiFlashReplica(t, dom, "test", "employee") var input Input var output Output - suiteData := plannercore.GetWindowPushDownSuiteData() + suiteData := GetWindowPushDownSuiteData() suiteData.LoadTestCases(t, &input, &output) testWithData(t, tk, input, output) } @@ -95,11 +80,11 @@ func TestWindowPushDownPlans(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists employee") tk.MustExec("create table employee (empid int, deptid int, salary decimal(10,2))") - SetTiFlashReplica(t, dom, "test", "employee") + internal.SetTiFlashReplica(t, dom, "test", "employee") var input Input var output Output - suiteData := plannercore.GetWindowPushDownSuiteData() + suiteData := GetWindowPushDownSuiteData() suiteData.LoadTestCases(t, &input, &output) testWithData(t, tk, input, output) } @@ -113,11 +98,11 @@ func TestWindowPlanWithOtherOperators(t *testing.T) { tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists employee") tk.MustExec("create table employee (empid int, deptid int, salary decimal(10,2))") - SetTiFlashReplica(t, dom, "test", "employee") + internal.SetTiFlashReplica(t, dom, "test", "employee") var input Input var output Output - suiteData := plannercore.GetWindowPushDownSuiteData() + suiteData := GetWindowPushDownSuiteData() suiteData.LoadTestCases(t, &input, &output) testWithData(t, tk, input, output) } @@ -131,8 +116,8 @@ func TestIssue34765(t *testing.T) { tk.MustExec("create table t1(c1 varchar(32), c2 datetime, c3 bigint, c4 varchar(64));") tk.MustExec("create table t2(b2 varchar(64));") tk.MustExec("set tidb_enforce_mpp=1;") - SetTiFlashReplica(t, dom, "test", "t1") - SetTiFlashReplica(t, dom, "test", "t2") + internal.SetTiFlashReplica(t, dom, "test", "t1") + internal.SetTiFlashReplica(t, dom, "test", "t2") require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/planner/core/CheckMPPWindowSchemaLength", "return")) defer func() { diff --git a/planner/core/expression_rewriter_test.go b/planner/core/expression_rewriter_test.go index 94a63d814c030..82dc1cef7b451 100644 --- a/planner/core/expression_rewriter_test.go +++ b/planner/core/expression_rewriter_test.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/tidb/parser/terror" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/testkit/testdata" "github.com/stretchr/testify/require" ) @@ -378,62 +377,3 @@ func TestInsertOnDuplicateLazyMoreThan1Row(t *testing.T) { tk.MustExec("INSERT INTO t2 (a) VALUES (1) ON DUPLICATE KEY UPDATE a= (SELECT b FROM source);") tk.MustExec("DROP TABLE if exists t1, t2, source;") } - -func TestMultiColInExpression(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)") - tk.MustExec("insert into t1 values(1,1),(2,null),(null,3),(4,4)") - tk.MustExec("analyze table t1") - tk.MustExec("create table t2(a int, b int)") - tk.MustExec("insert into t2 values(1,1),(2,null),(null,3),(5,4)") - tk.MustExec("analyze table t2") - var input []string - var output []struct { - SQL string - Plan []string - Res []string - } - - // Default RPC encoding may cause statistics explain result differ and then the test unstable. - tk.MustExec("set @@tidb_enable_chunk_rpc = on") - - expressionRewriterSuiteData := plannercore.GetExpressionRewriterSuiteData() - expressionRewriterSuiteData.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...)) - } -} - -func TestBitFuncsReturnType(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("set tidb_cost_model_version=2") - tk.MustExec("create table t (a timestamp, b varbinary(32))") - tk.MustExec("insert into t values ('2006-08-27 21:57:57', 0x373037343631313230)") - tk.MustExec("analyze table t") - var input []string - var output []struct { - Plan []string - } - - expressionRewriterSuiteData := plannercore.GetExpressionRewriterSuiteData() - expressionRewriterSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - } -} diff --git a/planner/core/internal/BUILD.bazel b/planner/core/internal/BUILD.bazel new file mode 100644 index 0000000000000..8d0fefac9394a --- /dev/null +++ b/planner/core/internal/BUILD.bazel @@ -0,0 +1,15 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "internal", + srcs = ["testkit.go"], + importpath = "github.com/pingcap/tidb/planner/core/internal", + visibility = ["//planner/core:__subpackages__"], + deps = [ + "//domain", + "//expression/aggregation", + "//parser/model", + "//sessionctx", + "@com_github_stretchr_testify//require", + ], +) diff --git a/planner/core/internal/testkit.go b/planner/core/internal/testkit.go new file mode 100644 index 0000000000000..ed335ab783466 --- /dev/null +++ b/planner/core/internal/testkit.go @@ -0,0 +1,50 @@ +// 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 internal + +import ( + "testing" + + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx" + "github.com/stretchr/testify/require" +) + +func SetTiFlashReplica(t *testing.T, dom *domain.Domain, dbName, tableName string) { + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr(dbName)) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == tableName { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } +} + +// WrapCastForAggFuncs wraps the args of an aggregate function with a cast function. +// If the mode is FinalMode or Partial2Mode, we do not need to wrap cast upon the args, +// since the types of the args are already the expected. +func WrapCastForAggFuncs(sctx sessionctx.Context, aggFuncs []*aggregation.AggFuncDesc) { + for i := range aggFuncs { + if aggFuncs[i].Mode != aggregation.FinalMode && aggFuncs[i].Mode != aggregation.Partial2Mode { + aggFuncs[i].WrapCastForAggArgs(sctx) + } + } +} diff --git a/planner/core/main_test.go b/planner/core/main_test.go index dc321fcd92921..efe59fd011686 100644 --- a/planner/core/main_test.go +++ b/planner/core/main_test.go @@ -18,44 +18,15 @@ import ( "flag" "testing" - "github.com/pingcap/tidb/testkit/testdata" - "github.com/pingcap/tidb/testkit/testmain" "github.com/pingcap/tidb/testkit/testsetup" "go.uber.org/goleak" ) -var testDataMap = make(testdata.BookKeeper) -var indexMergeSuiteData testdata.TestData -var planSuiteUnexportedData testdata.TestData - func TestMain(m *testing.M) { testsetup.SetupForCommonTest() flag.Parse() - testDataMap.LoadTestSuiteData("testdata", "integration_partition_suite") - testDataMap.LoadTestSuiteData("testdata", "index_merge_suite") - testDataMap.LoadTestSuiteData("testdata", "plan_normalized_suite") - testDataMap.LoadTestSuiteData("testdata", "stats_suite") - testDataMap.LoadTestSuiteData("testdata", "ordered_result_mode_suite") - testDataMap.LoadTestSuiteData("testdata", "point_get_plan") - testDataMap.LoadTestSuiteData("testdata", "enforce_mpp_suite") - testDataMap.LoadTestSuiteData("testdata", "expression_rewriter_suite") - testDataMap.LoadTestSuiteData("testdata", "partition_pruner") - testDataMap.LoadTestSuiteData("testdata", "plan_suite") - testDataMap.LoadTestSuiteData("testdata", "integration_suite") - testDataMap.LoadTestSuiteData("testdata", "analyze_suite") - testDataMap.LoadTestSuiteData("testdata", "window_push_down_suite") - testDataMap.LoadTestSuiteData("testdata", "plan_suite_unexported") - testDataMap.LoadTestSuiteData("testdata", "join_reorder_suite") - testDataMap.LoadTestSuiteData("testdata", "flat_plan_suite") - testDataMap.LoadTestSuiteData("testdata", "binary_plan_suite") - testDataMap.LoadTestSuiteData("testdata", "json_plan_suite") - testDataMap.LoadTestSuiteData("testdata", "derive_topn_from_window") - - indexMergeSuiteData = testDataMap["index_merge_suite"] - planSuiteUnexportedData = testDataMap["plan_suite_unexported"] - opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), @@ -65,82 +36,5 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - callback := func(i int) int { - testDataMap.GenerateOutputIfNeeded() - return i - } - - goleak.VerifyTestMain(testmain.WrapTestingM(m, callback), opts...) -} - -func GetIntegrationPartitionSuiteData() testdata.TestData { - return testDataMap["integration_partition_suite"] -} - -func GetPlanNormalizedSuiteData() testdata.TestData { - return testDataMap["plan_normalized_suite"] -} - -func GetStatsSuiteData() testdata.TestData { - return testDataMap["stats_suite"] -} - -func GetOrderedResultModeSuiteData() testdata.TestData { - return testDataMap["ordered_result_mode_suite"] -} - -func GetJoinReorderSuiteData() testdata.TestData { - return testDataMap["join_reorder_suite"] -} - -func GetPointGetPlanData() testdata.TestData { - return testDataMap["point_get_plan"] -} - -func GetEnforceMPPSuiteData() testdata.TestData { - return testDataMap["enforce_mpp_suite"] -} - -func GetExpressionRewriterSuiteData() testdata.TestData { - return testDataMap["expression_rewriter_suite"] -} - -func GetPartitionPrunerData() testdata.TestData { - return testDataMap["partition_pruner"] -} - -func GetPlanSuiteData() testdata.TestData { - return testDataMap["plan_suite"] -} - -func GetIntegrationSuiteData() testdata.TestData { - return testDataMap["integration_suite"] -} - -func GetAnalyzeSuiteData() testdata.TestData { - return testDataMap["analyze_suite"] -} - -func GetWindowPushDownSuiteData() testdata.TestData { - return testDataMap["window_push_down_suite"] -} - -func GetFlatPlanSuiteData() testdata.TestData { - return testDataMap["flat_plan_suite"] -} - -func GetBinaryPlanSuiteData() testdata.TestData { - return testDataMap["binary_plan_suite"] -} - -func GetIndexMergeSuiteData() testdata.TestData { - return testDataMap["index_merge_suite"] -} - -func GetJSONPlanSuiteData() testdata.TestData { - return testDataMap["json_plan_suite"] -} - -func GetDerivedTopNSuiteData() testdata.TestData { - return testDataMap["derive_topn_from_window"] + goleak.VerifyTestMain(m, opts...) } diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 6bfe3f6e04dea..b79332c146b52 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -16,9 +16,7 @@ package core_test import ( "bytes" - "encoding/json" "fmt" - "strings" "testing" "github.com/pingcap/failpoint" @@ -34,174 +32,11 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/plancodec" "github.com/stretchr/testify/require" ) -func TestPreferRangeScan(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(10) NOT NULL AUTO_INCREMENT,`name` varchar(50) NOT NULL DEFAULT 'tidb',`age` int(11) NOT NULL,`addr` varchar(50) DEFAULT 'The ocean of stars',PRIMARY KEY (`id`),KEY `idx_age` (`age`))") - tk.MustExec("insert into test(age) values(5);") - tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") - tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") - tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") - tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") - tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") - tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") - tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") - tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") - tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") - tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") - tk.MustExec("insert into test(name,age,addr) select name,age,addr from test;") - tk.MustExec("analyze table test;") - - // 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 - } - planNormalizedSuiteData := core.GetPlanNormalizedSuiteData() - planNormalizedSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - if i == 0 { - tk.MustExec("set session tidb_opt_prefer_range_scan=0") - } else if i == 1 { - tk.MustExec("set session tidb_opt_prefer_range_scan=1") - } - tk.Session().GetSessionVars().PlanID = 0 - tk.MustExec(tt) - info := tk.Session().ShowProcess() - require.NotNil(t, info) - p, ok := info.Plan.(core.Plan) - require.True(t, ok) - normalized, digest := core.NormalizePlan(p) - - // test the new normalization code - flat := core.FlattenPhysicalPlan(p, false) - newNormalized, newDigest := core.NormalizeFlatPlan(flat) - require.Equal(t, normalized, newNormalized) - require.Equal(t, digest, newDigest) - - normalizedPlan, err := plancodec.DecodeNormalizedPlan(normalized) - normalizedPlanRows := getPlanRows(normalizedPlan) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = normalizedPlanRows - }) - compareStringSlice(t, normalizedPlanRows, output[i].Plan) - } -} - -func TestNormalizedPlan(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set @@tidb_partition_prune_mode='static';") - tk.MustExec("drop table if exists t1,t2,t3,t4") - tk.MustExec("create table t1 (a int key,b int,c int, index (b));") - tk.MustExec("create table t2 (a int key,b int,c int, index (b));") - tk.MustExec("create table t3 (a int key,b int) partition by hash(a) partitions 2;") - tk.MustExec("create table t4 (a int, b int, index(a)) partition by range(a) (partition p0 values less than (10),partition p1 values less than MAXVALUE);") - tk.MustExec("set @@global.tidb_enable_foreign_key=1") - tk.MustExec("set @@foreign_key_checks=1") - tk.MustExec("create table t5 (id int key, id2 int, id3 int, unique index idx2(id2), index idx3(id3));") - tk.MustExec("create table t6 (id int, id2 int, id3 int, index idx_id(id), index idx_id2(id2), " + - "foreign key fk_1 (id) references t5(id) ON UPDATE CASCADE ON DELETE CASCADE, " + - "foreign key fk_2 (id2) references t5(id2) ON UPDATE CASCADE, " + - "foreign key fk_3 (id3) references t5(id3) ON DELETE CASCADE);") - tk.MustExec("insert into t5 values (1,1,1), (2,2,2)") - var input []string - var output []struct { - SQL string - Plan []string - } - planNormalizedSuiteData := core.GetPlanNormalizedSuiteData() - planNormalizedSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - tk.Session().GetSessionVars().PlanID = 0 - tk.MustExec(tt) - info := tk.Session().ShowProcess() - require.NotNil(t, info) - p, ok := info.Plan.(core.Plan) - require.True(t, ok) - normalized, digest := core.NormalizePlan(p) - - // test the new normalization code - flat := core.FlattenPhysicalPlan(p, false) - newNormalized, newDigest := core.NormalizeFlatPlan(flat) - require.Equal(t, normalized, newNormalized) - require.Equal(t, digest, newDigest) - // Test for GenHintsFromFlatPlan won't panic. - core.GenHintsFromFlatPlan(flat) - - normalizedPlan, err := plancodec.DecodeNormalizedPlan(normalized) - normalizedPlanRows := getPlanRows(normalizedPlan) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = normalizedPlanRows - }) - compareStringSlice(t, normalizedPlanRows, output[i].Plan) - } -} - -func TestNormalizedPlanForDiffStore(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1") - tk.MustExec("create table t1 (a int, b int, c int, primary key(a))") - tk.MustExec("insert into t1 values(1,1,1), (2,2,2), (3,3,3)") - tbl, 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. - tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} - - var input []string - var output []struct { - Digest string - Plan []string - } - planNormalizedSuiteData := core.GetPlanNormalizedSuiteData() - planNormalizedSuiteData.LoadTestCases(t, &input, &output) - lastDigest := "" - for i, tt := range input { - tk.Session().GetSessionVars().PlanID = 0 - tk.MustExec(tt) - info := tk.Session().ShowProcess() - require.NotNil(t, info) - ep, ok := info.Plan.(*core.Explain) - require.True(t, ok) - normalized, digest := core.NormalizePlan(ep.TargetPlan) - - // test the new normalization code - flat := core.FlattenPhysicalPlan(ep.TargetPlan, false) - newNormalized, newPlanDigest := core.NormalizeFlatPlan(flat) - require.Equal(t, digest, newPlanDigest) - require.Equal(t, normalized, newNormalized) - - normalizedPlan, err := plancodec.DecodeNormalizedPlan(normalized) - normalizedPlanRows := getPlanRows(normalizedPlan) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].Digest = digest.String() - output[i].Plan = normalizedPlanRows - }) - compareStringSlice(t, normalizedPlanRows, output[i].Plan) - require.NotEqual(t, digest.String(), lastDigest) - lastDigest = digest.String() - } -} - func TestEncodeDecodePlan(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -524,18 +359,6 @@ func testNormalizeDigest(tk *testkit.TestKit, t *testing.T, sql1, sql2 string, i } } -func getPlanRows(planStr string) []string { - planStr = strings.Replace(planStr, "\t", " ", -1) - return strings.Split(planStr, "\n") -} - -func compareStringSlice(t *testing.T, ss1, ss2 []string) { - require.Equal(t, len(ss1), len(ss2)) - for i, s := range ss1 { - require.Equal(t, len(s), len(ss2[i])) - } -} - func TestExplainFormatHint(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -1124,37 +947,6 @@ func TestOuterJoinOnNull(t *testing.T) { tk.MustQuery("SELECT * FROM t2 RIGHT JOIN t3 ON t2.c0 WHERE ((NOT ('i'))AND(t2.c0)) IS NULL;").Check(testkit.Rows(" 1")) } -func TestJSONPlanInExplain(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(id int, key(id))") - tk.MustExec("create table t2(id int, key(id))") - - var input []string - var output []struct { - SQL string - JSONPlan []*core.ExplainInfoForEncode - } - planSuiteData := core.GetJSONPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - - for i, test := range input { - resJSON := tk.MustQuery(test).Rows() - var res []*core.ExplainInfoForEncode - require.NoError(t, json.Unmarshal([]byte(resJSON[0][0].(string)), &res)) - for j, expect := range output[i].JSONPlan { - require.Equal(t, expect.ID, res[j].ID) - require.Equal(t, expect.EstRows, res[j].EstRows) - require.Equal(t, expect.ActRows, res[j].ActRows) - require.Equal(t, expect.TaskType, res[j].TaskType) - require.Equal(t, expect.AccessObject, res[j].AccessObject) - require.Equal(t, expect.OperatorInfo, res[j].OperatorInfo) - } - } -} - func TestIssue40535(t *testing.T) { store := testkit.CreateMockStore(t) var cfg kv.InjectionConfig diff --git a/planner/core/point_get_plan_test.go b/planner/core/point_get_plan_test.go index 812151afc26ec..1c6e9aa42a1b4 100644 --- a/planner/core/point_get_plan_test.go +++ b/planner/core/point_get_plan_test.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/testkit/testdata" dto "github.com/prometheus/client_model/go" "github.com/stretchr/testify/require" ) @@ -338,37 +337,6 @@ func TestPointGetId(t *testing.T) { } } -func TestCBOPointGet(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly - tk.MustExec("create table t (a varchar(20), b int, c int, d int, primary key(a), unique key(b, c))") - tk.MustExec("insert into t values('1',4,4,1), ('2',3,3,2), ('3',2,2,3), ('4',1,1,4)") - - var input []string - var output []struct { - SQL string - Plan []string - Res []string - } - pointGetPlanData := core.GetPointGetPlanData() - pointGetPlanData.LoadTestCases(t, &input, &output) - require.Equal(t, len(input), len(output)) - for i, sql := range input { - plan := tk.MustQuery("explain format = 'brief' " + sql) - res := tk.MustQuery(sql) - testdata.OnRecord(func() { - output[i].SQL = sql - output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) - output[i].Res = testdata.ConvertRowsToStrings(res.Rows()) - }) - plan.Check(testkit.Rows(output[i].Plan...)) - res.Check(testkit.Rows(output[i].Res...)) - } -} - func TestPartitionBatchPointGetPlanCache(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -841,46 +809,6 @@ func TestBatchPointGetWithInvisibleIndex(t *testing.T) { )) } -func TestCBOShouldNotUsePointGet(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("drop tables if exists t1, t2, t3, t4, t5") - tk.MustExec("create table t1(id varchar(20) primary key)") - tk.MustExec("create table t2(id varchar(20), unique(id))") - tk.MustExec("create table t3(id varchar(20), d varchar(20), unique(id, d))") - tk.MustExec("create table t4(id int, d varchar(20), c varchar(20), unique(id, d))") - tk.MustExec("create table t5(id bit(64) primary key)") - tk.MustExec("insert into t1 values('asdf'), ('1asdf')") - tk.MustExec("insert into t2 values('asdf'), ('1asdf')") - tk.MustExec("insert into t3 values('asdf', 't'), ('1asdf', 't')") - tk.MustExec("insert into t4 values(1, 'b', 'asdf'), (1, 'c', 'jkl'), (1, 'd', '1jkl')") - tk.MustExec("insert into t5 values(48)") - - var input []string - var output []struct { - SQL string - Plan []string - Res []string - } - - pointGetPlanData := core.GetPointGetPlanData() - pointGetPlanData.LoadTestCases(t, &input, &output) - require.Equal(t, len(input), len(output)) - for i, sql := range input { - plan := tk.MustQuery("explain format = 'brief' " + sql) - res := tk.MustQuery(sql) - testdata.OnRecord(func() { - output[i].SQL = sql - output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) - output[i].Res = testdata.ConvertRowsToStrings(res.Rows()) - }) - plan.Check(testkit.Rows(output[i].Plan...)) - res.Check(testkit.Rows(output[i].Res...)) - } -} - func TestPointGetWithIndexHints(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/planner/core/rule_inject_extra_projection.go b/planner/core/rule_inject_extra_projection.go index 308584b0bc1c9..9d5f86e41c0f6 100644 --- a/planner/core/rule_inject_extra_projection.go +++ b/planner/core/rule_inject_extra_projection.go @@ -20,8 +20,8 @@ import ( "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/core/internal" "github.com/pingcap/tidb/planner/util" - "github.com/pingcap/tidb/sessionctx" ) // InjectExtraProjection is used to extract the expressions of specific @@ -108,17 +108,6 @@ func injectProjBelowUnion(un *PhysicalUnionAll) *PhysicalUnionAll { return un } -// wrapCastForAggFunc wraps the args of an aggregate function with a cast function. -// If the mode is FinalMode or Partial2Mode, we do not need to wrap cast upon the args, -// since the types of the args are already the expected. -func wrapCastForAggFuncs(sctx sessionctx.Context, aggFuncs []*aggregation.AggFuncDesc) { - for i := range aggFuncs { - if aggFuncs[i].Mode != aggregation.FinalMode && aggFuncs[i].Mode != aggregation.Partial2Mode { - aggFuncs[i].WrapCastForAggArgs(sctx) - } - } -} - // InjectProjBelowAgg injects a ProjOperator below AggOperator. So that All // scalar functions in aggregation may speed up by vectorized evaluation in // the `proj`. If all the args of `aggFuncs`, and all the item of `groupByItems` @@ -126,7 +115,7 @@ func wrapCastForAggFuncs(sctx sessionctx.Context, aggFuncs []*aggregation.AggFun func InjectProjBelowAgg(aggPlan PhysicalPlan, aggFuncs []*aggregation.AggFuncDesc, groupByItems []expression.Expression) PhysicalPlan { hasScalarFunc := false - wrapCastForAggFuncs(aggPlan.SCtx(), aggFuncs) + internal.WrapCastForAggFuncs(aggPlan.SCtx(), aggFuncs) for i := 0; !hasScalarFunc && i < len(aggFuncs); i++ { for _, arg := range aggFuncs[i].Args { _, isScalarFunc := arg.(*expression.ScalarFunction) diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index 9fb0c7e83ab1f..37cb10164a4cc 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -17,157 +17,10 @@ package core_test import ( "testing" - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/parser/model" - plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/testkit/testdata" "github.com/stretchr/testify/require" ) -func runJoinReorderTestData(t *testing.T, tk *testkit.TestKit, name string) { - var input []string - var output []struct { - SQL string - Plan []string - Warning []string - } - joinReorderSuiteData := plannercore.GetJoinReorderSuiteData() - joinReorderSuiteData.LoadTestCasesByName(name, t, &input, &output) - require.Equal(t, len(input), len(output)) - for i := range input { - testdata.OnRecord(func() { - output[i].SQL = input[i] - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + input[i]).Rows()) - output[i].Warning = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) - }) - tk.MustQuery("explain format = 'brief' " + input[i]).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warning...)) - } -} - -func TestStraightJoinHint(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, t1, t2, t3, t4;") - tk.MustExec("create table t(a int, b int, key(a));") - tk.MustExec("create table t1(a int, b int, key(a));") - tk.MustExec("create table t2(a int, b int, key(a));") - tk.MustExec("create table t3(a int, b int, key(a));") - tk.MustExec("create table t4(a int, b int, key(a));") - runJoinReorderTestData(t, tk, "TestStraightJoinHint") -} - -func TestLeadingJoinHint(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, t1, t2, t3, t4, t5, t6, t7, t8;") - tk.MustExec("create table t(a int, b int, key(a));") - tk.MustExec("create table t1(a int, b int, key(a));") - tk.MustExec("create table t2(a int, b int, key(a));") - tk.MustExec("create table t3(a int, b int, key(a));") - tk.MustExec("create table t4(a int, b int, key(a));") - tk.MustExec("create table t5(a int, b int, key(a));") - tk.MustExec("create table t6(a int, b int, key(a));") - tk.MustExec("create table t7(a int, b int, key(a));") - tk.MustExec("create table t8(a int, b int, key(a));") - runJoinReorderTestData(t, tk, "TestLeadingJoinHint") - - // test cases for multiple leading hints - tk.MustExec("select /*+ leading(t1) leading(t2) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) -} - -func TestJoinOrderHint(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t, t1, t2, t3, t4, t5, t6, t7, t8;") - tk.MustExec("create table t(a int, b int, key(a));") - tk.MustExec("create table t1(a int, b int, key(a));") - tk.MustExec("create table t2(a int, b int, key(a));") - tk.MustExec("create table t3(a int, b int, key(a));") - tk.MustExec("create table t4(a int, b int, key(a));") - tk.MustExec("create table t5(a int, b int, key(a));") - tk.MustExec("create table t6(a int, b int, key(a));") - tk.MustExec("create table t7(a int, b int, key(a));") - tk.MustExec("create table t8(a int, b int, key(a));") - - // test cases for using the leading hint and straight_join hint at the same time - tk.MustExec("select /*+ leading(t1) straight_join() */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use the straight_join hint, when we use the leading hint and straight_join hint at the same time, all leading hints will be invalid")) - - tk.MustExec("select /*+ straight_join() leading(t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use the straight_join hint, when we use the leading hint and straight_join hint at the same time, all leading hints will be invalid")) - - // more join order hints appear in the same time - tk.MustExec("select /*+ leading(t1) leading(t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) - - tk.MustExec("select /*+ leading(t1) leading(t2) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) - - tk.MustExec("select /*+ straight_join() straight_join() */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 STRAIGHT_JOIN() is defined more than once, only the last definition takes effect")) - - // test cases for table name in hint - // the same table appears in the leading hint - tk.MustExec("select /*+ leading(t1, t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t1) in optimizer hint /*+ LEADING(t1, t1) */. Maybe you can use the table alias name", - "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) - - tk.MustExec("select /*+ leading(t1, t2, t1) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t1) in optimizer hint /*+ LEADING(t1, t2, t1) */. Maybe you can use the table alias name", - "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) - - // the wrong table appears in the leading hint - tk.MustExec("select /*+ leading(t) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t) in optimizer hint /*+ LEADING(t) */. Maybe you can use the table alias name")) - - tk.MustExec("select /*+ leading(t1, t2, t) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t) in optimizer hint /*+ LEADING(t1, t2, t) */. Maybe you can use the table alias name", - "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) - - // table alias in the leading hint - tk.MustExec("select /*+ leading(t) */ * from t1 t join t2 on t.a=t2.a join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows()) - - tk.MustExec("select /*+ leading(t1) */ * from t1 t join t2 on t.a=t2.a join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t1) in optimizer hint /*+ LEADING(t1) */. Maybe you can use the table alias name")) - - tk.MustExec("select /*+ leading(t2, t) */ * from t1 t join t2 on t.a=t2.a join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows()) - - tk.MustExec("select /*+ leading(t2, t1) */ * from t1 t join t2 on t.a=t2.a join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t1) in optimizer hint /*+ LEADING(t2, t1) */. Maybe you can use the table alias name", - "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) - - // table name in leading hint cross query block - // Todo: Can not handle this case yet. Because when we extract the join group, it will get the join group {t1, t2, t3}. - // So the table 't4' can not be used. - tk.MustExec("select /*+ leading(t4) */ * from (select t2.b from t1 join t2 on t1.a=t2.a) t4 join t3 on t4.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) - - tk.MustExec("select /*+ leading(t3, t2@sel_2) */ * from (select t2.b from t1 join t2 on t1.a=t2.a) t4 join t3 on t4.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t2) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name")) - - tk.MustExec("select * from (select /*+ leading(t1, t3@sel_1) */ t2.b from t1 join t2 on t1.a=t2.a) t4 join t3 on t4.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t1, t3) */. Maybe you can use the table alias name")) - - tk.MustExec("select /*+ leading(t3) */ * from (select /*+ leading(t1) */ t2.b from t1 join t2 on t1.a=t2.a) t4 join t3 on t4.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) - - runJoinReorderTestData(t, tk, "TestJoinOrderHint") -} - func TestJoinOrderHintWithBinding(t *testing.T) { store := testkit.CreateMockStore(t) @@ -216,143 +69,6 @@ func TestJoinOrderHintWithBinding(t *testing.T) { tk.MustExec("drop global binding for select * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") } -func TestJoinOrderHint4StaticPartitionTable(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, t1, t2, t3;") - tk.MustExec(`create table t(a int, b int) partition by hash(a) partitions 3`) - tk.MustExec(`create table t1(a int, b int) partition by hash(a) partitions 4`) - tk.MustExec(`create table t2(a int, b int) partition by hash(a) partitions 5`) - tk.MustExec(`create table t3(a int, b int) partition by hash(b) partitions 3`) - tk.MustExec(`create table t4(a int, b int) partition by hash(a) partitions 4`) - tk.MustExec(`create table t5(a int, b int) partition by hash(a) partitions 5`) - tk.MustExec(`create table t6(a int, b int) partition by hash(b) partitions 3`) - - tk.MustExec(`set @@tidb_partition_prune_mode="static"`) - tk.MustExec("set @@tidb_enable_outer_join_reorder=true") - runJoinReorderTestData(t, tk, "TestJoinOrderHint4StaticPartitionTable") -} - -func TestJoinOrderHint4DynamicPartitionTable(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, t1, t2, t3;") - tk.MustExec(`create table t(a int, b int) partition by hash(a) partitions 3`) - tk.MustExec(`create table t1(a int, b int) partition by hash(a) partitions 4`) - tk.MustExec(`create table t2(a int, b int) partition by hash(a) partitions 5`) - tk.MustExec(`create table t3(a int, b int) partition by hash(b) partitions 3`) - tk.MustExec(`create table t4(a int, b int) partition by hash(a) partitions 4`) - tk.MustExec(`create table t5(a int, b int) partition by hash(a) partitions 5`) - tk.MustExec(`create table t6(a int, b int) partition by hash(b) partitions 3`) - - tk.MustExec(`set @@tidb_partition_prune_mode="dynamic"`) - tk.MustExec("set @@tidb_enable_outer_join_reorder=true") - runJoinReorderTestData(t, tk, "TestJoinOrderHint4DynamicPartitionTable") -} - -func TestJoinOrderHint4DifferentJoinType(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, t1, t2, t3, t4, t5, t6, t7, t8;") - tk.MustExec("create table t(a int, b int, key(a));") - tk.MustExec("create table t1(a int, b int, key(a));") - tk.MustExec("create table t2(a int, b int, key(a));") - tk.MustExec("create table t3(a int, b int, key(a));") - tk.MustExec("create table t4(a int, b int, key(a));") - tk.MustExec("create table t5(a int, b int, key(a));") - tk.MustExec("create table t6(a int, b int, key(a));") - tk.MustExec("create table t7(a int, b int, key(a));") - tk.MustExec("create table t8(a int, b int, key(a));") - tk.MustExec("set @@tidb_enable_outer_join_reorder=true") - - runJoinReorderTestData(t, tk, "TestJoinOrderHint4DifferentJoinType") -} - -func TestJoinOrderHint4TiFlash(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t, t1, t2, t3;") - tk.MustExec("create table t(a int, b int, key(a));") - tk.MustExec("create table t1(a int, b int, key(a));") - tk.MustExec("create table t2(a int, b int, key(a));") - tk.MustExec("create table t3(a int, b int, key(a));") - tk.MustExec("create table t4(a int, b int, key(a));") - tk.MustExec("create table t5(a int, b int, key(a));") - tk.MustExec("create table t6(a int, b int, key(a));") - tk.MustExec("set @@tidb_enable_outer_join_reorder=true") - - // 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 { - tableName := tblInfo.Name.L - if tableName == "t" || tableName == "t1" || tableName == "t2" || tableName == "t3" || tableName == "t4" || tableName == "t5" || tableName == "t6" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") - runJoinReorderTestData(t, tk, "TestJoinOrderHint4TiFlash") -} - -func TestJoinOrderHint4Subquery(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, t1, t2, t3, t4, t5, t6, t7, t8;") - tk.MustExec("create table t(a int, b int, key(a));") - tk.MustExec("create table t1(a int, b int, key(a));") - tk.MustExec("create table t2(a int, b int, key(a));") - tk.MustExec("create table t3(a int, b int, key(a));") - tk.MustExec("create table t4(a int, b int, key(a));") - tk.MustExec("create table t5(a int, b int, key(a));") - tk.MustExec("create table t6(a int, b int, key(a));") - tk.MustExec("create table t7(a int, b int, key(a));") - tk.MustExec("create table t8(a int, b int, key(a));") - tk.MustExec("insert into t3 values(1, 1), (2, 2), (3, 3);") - tk.MustExec("analyze table t3;") - - runJoinReorderTestData(t, tk, "TestJoinOrderHint4Subquery") -} - -func TestLeadingJoinHint4OuterJoin(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, t1, t2, t3, t4, t5, t6, t7, t8;") - tk.MustExec("create table t(a int, b int, key(a));") - tk.MustExec("create table t1(a int, b int, key(a));") - tk.MustExec("create table t2(a int, b int, key(a));") - tk.MustExec("create table t3(a int, b int, key(a));") - tk.MustExec("create table t4(a int, b int, key(a));") - tk.MustExec("create table t5(a int, b int, key(a));") - tk.MustExec("create table t6(a int, b int, key(a));") - tk.MustExec("create table t7(a int, b int, key(a));") - tk.MustExec("create table t8(a int, b int, key(a));") - tk.MustExec("set @@tidb_enable_outer_join_reorder=true") - runJoinReorderTestData(t, tk, "TestLeadingJoinHint4OuterJoin") -} - func TestOuterJoinWIthEqCondCrossInnerJoin(t *testing.T) { store := testkit.CreateMockStore(t) diff --git a/planner/core/rule_result_reorder_test.go b/planner/core/rule_result_reorder_test.go index 1a178cd66bfe7..7f6c0201182cb 100644 --- a/planner/core/rule_result_reorder_test.go +++ b/planner/core/rule_result_reorder_test.go @@ -15,13 +15,10 @@ package core_test import ( - "fmt" "testing" - plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/testkit/testdata" "github.com/stretchr/testify/require" ) @@ -85,110 +82,6 @@ func TestClusteredIndex(t *testing.T) { tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOff } -func runTestData(t *testing.T, tk *testkit.TestKit, name string) { - var input []string - var output []struct { - Plan []string - } - statsSuiteData := plannercore.GetOrderedResultModeSuiteData() - statsSuiteData.LoadTestCasesByName(name, t, &input, &output) - require.Equal(t, len(input), len(output)) - for i := range input { - testdata.OnRecord(func() { - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + input[i]).Rows()) - }) - tk.MustQuery("explain " + input[i]).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestOrderedResultMode(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("set tidb_enable_ordered_result_mode=1") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a int primary key, b int, c int, d int, key(b))") - runTestData(t, tk, "TestOrderedResultMode") -} - -func TestOrderedResultModeOnDML(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_enable_ordered_result_mode=1") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a int primary key, b int, c int, key(b))") - runTestData(t, tk, "TestOrderedResultModeOnDML") -} - -func TestOrderedResultModeOnSubQuery(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_enable_ordered_result_mode=1") - tk.MustExec("drop table if exists t1") - tk.MustExec("drop table if exists t2") - tk.MustExec("create table t1 (a int primary key, b int, c int, d int, key(b))") - tk.MustExec("create table t2 (a int primary key, b int, c int, d int, key(b))") - runTestData(t, tk, "TestOrderedResultModeOnSubQuery") -} - -func TestOrderedResultModeOnJoin(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_enable_ordered_result_mode=1") - tk.MustExec("drop table if exists t1") - tk.MustExec("drop table if exists t2") - tk.MustExec("create table t1 (a int primary key, b int, c int, d int, key(b))") - tk.MustExec("create table t2 (a int primary key, b int, c int, d int, key(b))") - tk.MustExec("set @@tidb_enable_outer_join_reorder=true") - runTestData(t, tk, "TestOrderedResultModeOnJoin") -} - -func TestOrderedResultModeOnOtherOperators(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_enable_ordered_result_mode=1") - tk.MustExec("drop table if exists t1") - tk.MustExec("drop table if exists t2") - tk.MustExec("create table t1 (a int primary key, b int, c int, d int, unique key(b))") - tk.MustExec("create table t2 (a int primary key, b int, c int, d int, unique key(b))") - runTestData(t, tk, "TestOrderedResultModeOnOtherOperators") -} - -func TestOrderedResultModeOnPartitionTable(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(fmt.Sprintf(`set tidb_partition_prune_mode='%v'`, variable.DefTiDBPartitionPruneMode)) - tk.MustExec("set tidb_enable_ordered_result_mode=1") - tk.MustExec("drop table if exists thash") - tk.MustExec("drop table if exists trange") - tk.MustExec("create table thash (a int primary key, b int, c int, d int) partition by hash(a) partitions 4") - tk.MustExec(`create table trange (a int primary key, b int, c int, d int) partition by range(a) ( - partition p0 values less than (100), - partition p1 values less than (200), - partition p2 values less than (300), - partition p3 values less than (400))`) - tk.MustExec(`analyze table thash`) - tk.MustExec(`analyze table trange`) - tk.MustQuery("select @@tidb_partition_prune_mode").Check(testkit.Rows("dynamic")) - runTestData(t, tk, "TestOrderedResultModeOnPartitionTable") -} - func TestStableResultSwitch(t *testing.T) { store := testkit.CreateMockStore(t) From e2b4529e7561fd716fd7a4acc891480d2a4c8e9e Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 15 Feb 2023 00:23:59 +0800 Subject: [PATCH 02/12] *: improve planner/core to make test faster Signed-off-by: Weizhen Wang --- planner/core/BUILD.bazel | 2 +- planner/core/casetest/main_test.go | 11 - .../core/collect_column_stats_usage_test.go | 2 +- planner/core/logical_plan_test.go | 2171 ---------------- planner/core/logical_plans_test.go | 2241 ++++++++++++++++- planner/core/main_test.go | 22 +- .../testdata/plan_suite_unexported_in.json | 0 .../testdata/plan_suite_unexported_out.json | 0 8 files changed, 2130 insertions(+), 2319 deletions(-) delete mode 100644 planner/core/logical_plan_test.go rename planner/core/{casetest => }/testdata/plan_suite_unexported_in.json (100%) rename planner/core/{casetest => }/testdata/plan_suite_unexported_out.json (100%) diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index fdb55dcac7522..82883f76f2df6 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -177,7 +177,6 @@ go_test( "indexmerge_test.go", "integration_partition_test.go", "integration_test.go", - "logical_plan_test.go", "logical_plan_trace_test.go", "logical_plans_test.go", "main_test.go", @@ -248,6 +247,7 @@ go_test( "//testkit/ddlhelper", "//testkit/external", "//testkit/testdata", + "//testkit/testmain", "//testkit/testsetup", "//testkit/testutil", "//types", diff --git a/planner/core/casetest/main_test.go b/planner/core/casetest/main_test.go index 610faceee9bd2..8f4c1a1351f0e 100644 --- a/planner/core/casetest/main_test.go +++ b/planner/core/casetest/main_test.go @@ -25,8 +25,6 @@ import ( ) var testDataMap = make(testdata.BookKeeper) -var indexMergeSuiteData testdata.TestData -var planSuiteUnexportedData testdata.TestData func TestMain(m *testing.M) { testsetup.SetupForCommonTest() @@ -34,7 +32,6 @@ func TestMain(m *testing.M) { flag.Parse() testDataMap.LoadTestSuiteData("testdata", "integration_partition_suite") - testDataMap.LoadTestSuiteData("testdata", "index_merge_suite") testDataMap.LoadTestSuiteData("testdata", "plan_normalized_suite") testDataMap.LoadTestSuiteData("testdata", "stats_suite") testDataMap.LoadTestSuiteData("testdata", "ordered_result_mode_suite") @@ -46,16 +43,12 @@ func TestMain(m *testing.M) { testDataMap.LoadTestSuiteData("testdata", "integration_suite") testDataMap.LoadTestSuiteData("testdata", "analyze_suite") testDataMap.LoadTestSuiteData("testdata", "window_push_down_suite") - testDataMap.LoadTestSuiteData("testdata", "plan_suite_unexported") testDataMap.LoadTestSuiteData("testdata", "join_reorder_suite") testDataMap.LoadTestSuiteData("testdata", "flat_plan_suite") testDataMap.LoadTestSuiteData("testdata", "binary_plan_suite") testDataMap.LoadTestSuiteData("testdata", "json_plan_suite") testDataMap.LoadTestSuiteData("testdata", "derive_topn_from_window") - indexMergeSuiteData = testDataMap["index_merge_suite"] - planSuiteUnexportedData = testDataMap["plan_suite_unexported"] - opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), @@ -133,10 +126,6 @@ func GetBinaryPlanSuiteData() testdata.TestData { return testDataMap["binary_plan_suite"] } -func GetIndexMergeSuiteData() testdata.TestData { - return testDataMap["index_merge_suite"] -} - func GetJSONPlanSuiteData() testdata.TestData { return testDataMap["json_plan_suite"] } diff --git a/planner/core/collect_column_stats_usage_test.go b/planner/core/collect_column_stats_usage_test.go index 249f210050c56..82f40a834bf59 100644 --- a/planner/core/collect_column_stats_usage_test.go +++ b/planner/core/collect_column_stats_usage_test.go @@ -324,7 +324,7 @@ func TestCollectHistNeededColumns(t *testing.T) { }, } - s := createPlannerSuite() + s := casetest.createPlannerSuite() ctx := context.Background() for _, tt := range tests { comment := fmt.Sprintf("sql: %s", tt.sql) diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go deleted file mode 100644 index 82786703338f4..0000000000000 --- a/planner/core/logical_plan_test.go +++ /dev/null @@ -1,2171 +0,0 @@ -// Copyright 2015 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 core - -import ( - "context" - "fmt" - "sort" - "strings" - "testing" - - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/parser/ast" - "github.com/pingcap/tidb/parser/format" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/planner/property" - "github.com/pingcap/tidb/planner/util" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/testkit/testdata" - "github.com/pingcap/tidb/util/hint" - "github.com/stretchr/testify/require" -) - -type plannerSuite struct { - p *parser.Parser - is infoschema.InfoSchema - ctx sessionctx.Context -} - -func createPlannerSuite() (s *plannerSuite) { - s = new(plannerSuite) - tblInfos := []*model.TableInfo{ - MockSignedTable(), - MockUnsignedTable(), - MockView(), - MockNoPKTable(), - MockRangePartitionTable(), - MockHashPartitionTable(), - MockListPartitionTable(), - MockStateNoneColumnTable(), - } - id := int64(0) - for _, tblInfo := range tblInfos { - tblInfo.ID = id - id += 1 - pi := tblInfo.GetPartitionInfo() - if pi == nil { - continue - } - for _, def := range pi.Definitions { - def.ID = id - id += 1 - } - } - s.is = infoschema.MockInfoSchema(tblInfos) - s.ctx = MockContext() - domain.GetDomain(s.ctx).MockInfoCacheAndLoadInfoSchema(s.is) - s.ctx.GetSessionVars().EnableWindowFunction = true - s.p = parser.New() - s.p.SetParserConfig(parser.ParserConfig{EnableWindowFunction: true, EnableStrictDoubleTypeCheck: true}) - return -} - -func TestPredicatePushDown(t *testing.T) { - var input, output []string - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - s := createPlannerSuite() - ctx := context.Background() - for ith, ca := range input { - comment := fmt.Sprintf("for %s", ca) - stmt, err := s.p.ParseOneStmt(ca, "", "") - require.NoError(t, err, comment) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err) - p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) - require.NoError(t, err) - testdata.OnRecord(func() { - output[ith] = ToString(p) - }) - require.Equal(t, output[ith], ToString(p), fmt.Sprintf("for %s %d", ca, ith)) - } -} - -// Issue: 31399 -func TestImplicitCastNotNullFlag(t *testing.T) { - ctx := context.Background() - ca := "select count(*) from t3 group by a having bit_and(b) > 1;" - comment := fmt.Sprintf("for %s", ca) - s := createPlannerSuite() - stmt, err := s.p.ParseOneStmt(ca, "", "") - require.NoError(t, err, comment) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err) - p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagJoinReOrder|flagPrunColumns|flagEliminateProjection, p.(LogicalPlan)) - require.NoError(t, err) - // AggFuncs[0] is count; AggFuncs[1] is bit_and, args[0] is return type of the implicit cast - castNotNullFlag := (p.(*LogicalProjection).children[0].(*LogicalSelection).children[0].(*LogicalAggregation).AggFuncs[1].Args[0].GetType().GetFlag()) & mysql.NotNullFlag - var nullableFlag uint = 0 - require.Equal(t, nullableFlag, castNotNullFlag) -} - -func TestEliminateProjectionUnderUnion(t *testing.T) { - ctx := context.Background() - ca := "Select a from t3 join ( (select 127 as IDD from t3) union all (select 1 as IDD from t3) ) u on t3.b = u.IDD;" - comment := fmt.Sprintf("for %s", ca) - s := createPlannerSuite() - stmt, err := s.p.ParseOneStmt(ca, "", "") - require.NoError(t, err, comment) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err) - p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagJoinReOrder|flagPrunColumns|flagEliminateProjection, p.(LogicalPlan)) - require.NoError(t, err) - // after folding constants, the null flag should keep the same with the old one's (i.e., the schema's). - schemaNullFlag := p.(*LogicalProjection).children[0].(*LogicalJoin).children[1].Children()[1].(*LogicalProjection).schema.Columns[0].RetType.GetFlag() & mysql.NotNullFlag - exprNullFlag := p.(*LogicalProjection).children[0].(*LogicalJoin).children[1].Children()[1].(*LogicalProjection).Exprs[0].GetType().GetFlag() & mysql.NotNullFlag - require.Equal(t, exprNullFlag, schemaNullFlag) -} - -func TestJoinPredicatePushDown(t *testing.T) { - var ( - input []string - output []struct { - Left string - Right string - } - ) - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - - s := createPlannerSuite() - ctx := context.Background() - for i, ca := range input { - comment := fmt.Sprintf("for %s", ca) - stmt, err := s.p.ParseOneStmt(ca, "", "") - require.NoError(t, err, comment) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err, comment) - p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) - require.NoError(t, err, comment) - proj, ok := p.(*LogicalProjection) - require.True(t, ok, comment) - join, ok := proj.children[0].(*LogicalJoin) - require.True(t, ok, comment) - leftPlan, ok := join.children[0].(*DataSource) - require.True(t, ok, comment) - rightPlan, ok := join.children[1].(*DataSource) - require.True(t, ok, comment) - leftCond := fmt.Sprintf("%s", leftPlan.pushedDownConds) - rightCond := fmt.Sprintf("%s", rightPlan.pushedDownConds) - testdata.OnRecord(func() { - output[i].Left, output[i].Right = leftCond, rightCond - }) - require.Equal(t, output[i].Left, leftCond, comment) - require.Equal(t, output[i].Right, rightCond, comment) - } -} - -func TestOuterWherePredicatePushDown(t *testing.T) { - var ( - input []string - output []struct { - Sel string - Left string - Right string - } - ) - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - - s := createPlannerSuite() - ctx := context.Background() - for i, ca := range input { - comment := fmt.Sprintf("for %s", ca) - stmt, err := s.p.ParseOneStmt(ca, "", "") - require.NoError(t, err, comment) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err, comment) - p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) - require.NoError(t, err, comment) - proj, ok := p.(*LogicalProjection) - require.True(t, ok, comment) - selection, ok := proj.children[0].(*LogicalSelection) - require.True(t, ok, comment) - selCond := fmt.Sprintf("%s", selection.Conditions) - testdata.OnRecord(func() { - output[i].Sel = selCond - }) - require.Equal(t, output[i].Sel, selCond, comment) - join, ok := selection.children[0].(*LogicalJoin) - require.True(t, ok, comment) - leftPlan, ok := join.children[0].(*DataSource) - require.True(t, ok, comment) - rightPlan, ok := join.children[1].(*DataSource) - require.True(t, ok, comment) - leftCond := fmt.Sprintf("%s", leftPlan.pushedDownConds) - rightCond := fmt.Sprintf("%s", rightPlan.pushedDownConds) - testdata.OnRecord(func() { - output[i].Left, output[i].Right = leftCond, rightCond - }) - require.Equal(t, output[i].Left, leftCond, comment) - require.Equal(t, output[i].Right, rightCond, comment) - } -} - -func TestSimplifyOuterJoin(t *testing.T) { - var ( - input []string - output []struct { - Best string - JoinType string - } - ) - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - - s := createPlannerSuite() - ctx := context.Background() - for i, ca := range input { - comment := fmt.Sprintf("for %s", ca) - stmt, err := s.p.ParseOneStmt(ca, "", "") - require.NoError(t, err, comment) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err, comment) - p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) - require.NoError(t, err, comment) - planString := ToString(p) - testdata.OnRecord(func() { - output[i].Best = planString - }) - require.Equal(t, output[i].Best, planString, comment) - join, ok := p.(LogicalPlan).Children()[0].(*LogicalJoin) - if !ok { - join, ok = p.(LogicalPlan).Children()[0].Children()[0].(*LogicalJoin) - require.True(t, ok, comment) - } - testdata.OnRecord(func() { - output[i].JoinType = join.JoinType.String() - }) - require.Equal(t, output[i].JoinType, join.JoinType.String(), comment) - } -} - -func TestAntiSemiJoinConstFalse(t *testing.T) { - tests := []struct { - sql string - best string - joinType string - }{ - { - sql: "select a from t t1 where not exists (select a from t t2 where t1.a = t2.a and t2.b = 1 and t2.b = 2)", - best: "Join{DataScan(t1)->DataScan(t2)}(test.t.a,test.t.a)->Projection", - joinType: "anti semi join", - }, - } - - s := createPlannerSuite() - ctx := context.Background() - for _, ca := range tests { - comment := fmt.Sprintf("for %s", ca.sql) - stmt, err := s.p.ParseOneStmt(ca.sql, "", "") - require.NoError(t, err, comment) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err, comment) - p, err = logicalOptimize(context.TODO(), flagDecorrelate|flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) - require.NoError(t, err, comment) - require.Equal(t, ca.best, ToString(p), comment) - join, _ := p.(LogicalPlan).Children()[0].(*LogicalJoin) - require.Equal(t, ca.joinType, join.JoinType.String(), comment) - } -} - -func TestDeriveNotNullConds(t *testing.T) { - var ( - input []string - output []struct { - Plan string - Left string - Right string - } - ) - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - - s := createPlannerSuite() - ctx := context.Background() - for i, ca := range input { - comment := fmt.Sprintf("for %s", ca) - stmt, err := s.p.ParseOneStmt(ca, "", "") - require.NoError(t, err, comment) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err, comment) - p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain|flagDecorrelate, p.(LogicalPlan)) - require.NoError(t, err, comment) - testdata.OnRecord(func() { - output[i].Plan = ToString(p) - }) - require.Equal(t, output[i].Plan, ToString(p), comment) - join := p.(LogicalPlan).Children()[0].(*LogicalJoin) - left := join.Children()[0].(*DataSource) - right := join.Children()[1].(*DataSource) - leftConds := fmt.Sprintf("%s", left.pushedDownConds) - rightConds := fmt.Sprintf("%s", right.pushedDownConds) - testdata.OnRecord(func() { - output[i].Left, output[i].Right = leftConds, rightConds - }) - require.Equal(t, output[i].Left, leftConds, comment) - require.Equal(t, output[i].Right, rightConds, comment) - } -} - -func TestExtraPKNotNullFlag(t *testing.T) { - sql := "select count(*) from t3" - s := createPlannerSuite() - ctx := context.Background() - comment := fmt.Sprintf("for %s", sql) - stmt, err := s.p.ParseOneStmt(sql, "", "") - require.NoError(t, err, comment) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err, comment) - ds := p.(*LogicalProjection).children[0].(*LogicalAggregation).children[0].(*DataSource) - require.Equal(t, "_tidb_rowid", ds.Columns[2].Name.L) - require.Equal(t, mysql.PriKeyFlag|mysql.NotNullFlag, ds.Columns[2].GetFlag()) - require.Equal(t, mysql.PriKeyFlag|mysql.NotNullFlag, ds.schema.Columns[2].RetType.GetFlag()) -} - -func buildLogicPlan4GroupBy(s *plannerSuite, t *testing.T, sql string) (Plan, error) { - sqlMode := s.ctx.GetSessionVars().SQLMode - mockedTableInfo := MockSignedTable() - // mock the table info here for later use - // enable only full group by - s.ctx.GetSessionVars().SQLMode = sqlMode | mysql.ModeOnlyFullGroupBy - defer func() { s.ctx.GetSessionVars().SQLMode = sqlMode }() // restore it - comment := fmt.Sprintf("for %s", sql) - stmt, err := s.p.ParseOneStmt(sql, "", "") - require.NoError(t, err, comment) - - stmt.(*ast.SelectStmt).From.TableRefs.Left.(*ast.TableSource).Source.(*ast.TableName).TableInfo = mockedTableInfo - - p, _, err := BuildLogicalPlanForTest(context.Background(), s.ctx, stmt, s.is) - return p, err -} - -func TestGroupByWhenNotExistCols(t *testing.T) { - sqlTests := []struct { - sql string - expectedErrMatch string - }{ - { - sql: "select a from t group by b", - expectedErrMatch: ".*contains nonaggregated column 'test\\.t\\.a'.*", - }, - { - // has an as column alias - sql: "select a as tempField from t group by b", - expectedErrMatch: ".*contains nonaggregated column 'test\\.t\\.a'.*", - }, - { - // has as table alias - sql: "select tempTable.a from t as tempTable group by b", - expectedErrMatch: ".*contains nonaggregated column 'test\\.tempTable\\.a'.*", - }, - { - // has a func call - sql: "select length(a) from t group by b", - expectedErrMatch: ".*contains nonaggregated column 'test\\.t\\.a'.*", - }, - { - // has a func call with two cols - sql: "select length(b + a) from t group by b", - expectedErrMatch: ".*contains nonaggregated column 'test\\.t\\.a'.*", - }, - { - // has a func call with two cols - sql: "select length(a + b) from t group by b", - expectedErrMatch: ".*contains nonaggregated column 'test\\.t\\.a'.*", - }, - { - // has a func call with two cols - sql: "select length(a + b) as tempField from t group by b", - expectedErrMatch: ".*contains nonaggregated column 'test\\.t\\.a'.*", - }, - } - s := createPlannerSuite() - for _, test := range sqlTests { - sql := test.sql - p, err := buildLogicPlan4GroupBy(s, t, sql) - require.Nil(t, p) - require.Error(t, err) - require.Regexp(t, test.expectedErrMatch, err.Error()) - } -} - -func TestDupRandJoinCondsPushDown(t *testing.T) { - sql := "select * from t as t1 join t t2 on t1.a > rand() and t1.a > rand()" - comment := fmt.Sprintf("for %s", sql) - s := createPlannerSuite() - stmt, err := s.p.ParseOneStmt(sql, "", "") - require.NoError(t, err, comment) - p, _, err := BuildLogicalPlanForTest(context.Background(), s.ctx, stmt, s.is) - require.NoError(t, err, comment) - p, err = logicalOptimize(context.TODO(), flagPredicatePushDown, p.(LogicalPlan)) - require.NoError(t, err, comment) - proj, ok := p.(*LogicalProjection) - require.True(t, ok, comment) - join, ok := proj.children[0].(*LogicalJoin) - require.True(t, ok, comment) - leftPlan, ok := join.children[0].(*LogicalSelection) - require.True(t, ok, comment) - leftCond := fmt.Sprintf("%s", leftPlan.Conditions) - // Condition with mutable function cannot be de-duplicated when push down join conds. - require.Equal(t, "[gt(cast(test.t.a, double BINARY), rand()) gt(cast(test.t.a, double BINARY), rand())]", leftCond, comment) -} - -func TestTablePartition(t *testing.T) { - variable.EnableMDL.Store(false) - definitions := []model.PartitionDefinition{ - { - ID: 41, - Name: model.NewCIStr("p1"), - LessThan: []string{"16"}, - }, - { - ID: 42, - Name: model.NewCIStr("p2"), - LessThan: []string{"32"}, - }, - { - ID: 43, - Name: model.NewCIStr("p3"), - LessThan: []string{"64"}, - }, - { - ID: 44, - Name: model.NewCIStr("p4"), - LessThan: []string{"128"}, - }, - { - ID: 45, - Name: model.NewCIStr("p5"), - LessThan: []string{"maxvalue"}, - }, - } - is := MockPartitionInfoSchema(definitions) - // is1 equals to is without maxvalue partition. - definitions1 := make([]model.PartitionDefinition, len(definitions)-1) - copy(definitions1, definitions) - is1 := MockPartitionInfoSchema(definitions1) - isChoices := []infoschema.InfoSchema{is, is1} - - var ( - input []struct { - SQL string - IsIdx int - } - output []string - ) - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - - s := createPlannerSuite() - ctx := context.Background() - for i, ca := range input { - comment := fmt.Sprintf("for %s", ca.SQL) - stmt, err := s.p.ParseOneStmt(ca.SQL, "", "") - require.NoError(t, err, comment) - testdata.OnRecord(func() { - - }) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, isChoices[ca.IsIdx]) - require.NoError(t, err) - p, err = logicalOptimize(context.TODO(), flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain|flagPredicatePushDown|flagPartitionProcessor, p.(LogicalPlan)) - require.NoError(t, err) - planString := ToString(p) - testdata.OnRecord(func() { - output[i] = planString - }) - require.Equal(t, output[i], ToString(p), fmt.Sprintf("for %v", ca)) - } -} - -func TestSubquery(t *testing.T) { - var input, output []string - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - - s := createPlannerSuite() - ctx := context.Background() - for ith, ca := range input { - comment := fmt.Sprintf("for %s", ca) - stmt, err := s.p.ParseOneStmt(ca, "", "") - require.NoError(t, err, comment) - - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err) - if lp, ok := p.(LogicalPlan); ok { - p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain|flagSemiJoinRewrite, lp) - require.NoError(t, err) - } - testdata.OnRecord(func() { - output[ith] = ToString(p) - }) - require.Equal(t, output[ith], ToString(p), fmt.Sprintf("for %s %d", ca, ith)) - } -} - -func TestPlanBuilder(t *testing.T) { - var input, output []string - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - - s := createPlannerSuite() - s.ctx.GetSessionVars().CostModelVersion = modelVer1 - ctx := context.Background() - for i, ca := range input { - comment := fmt.Sprintf("for %s", ca) - stmt, err := s.p.ParseOneStmt(ca, "", "") - require.NoError(t, err, comment) - - s.ctx.GetSessionVars().SetHashJoinConcurrency(1) - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err) - if lp, ok := p.(LogicalPlan); ok { - p, err = logicalOptimize(context.TODO(), flagPrunColumns|flagPrunColumnsAgain, lp) - require.NoError(t, err) - } - testdata.OnRecord(func() { - output[i] = ToString(p) - }) - require.Equal(t, output[i], ToString(p), fmt.Sprintf("for %s", ca)) - } -} - -func TestJoinReOrder(t *testing.T) { - var input, output []string - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - - s := createPlannerSuite() - ctx := context.Background() - for i, tt := range input { - comment := fmt.Sprintf("for %s", tt) - stmt, err := s.p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err) - p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagJoinReOrder, p.(LogicalPlan)) - require.NoError(t, err) - planString := ToString(p) - testdata.OnRecord(func() { - output[i] = planString - }) - require.Equal(t, output[i], planString, fmt.Sprintf("for %s", tt)) - } -} - -func TestEagerAggregation(t *testing.T) { - var input []string - var output []string - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - - s := createPlannerSuite() - ctx := context.Background() - s.ctx.GetSessionVars().AllowAggPushDown = true - defer func() { - s.ctx.GetSessionVars().AllowAggPushDown = false - }() - for ith, tt := range input { - comment := fmt.Sprintf("for %s", tt) - stmt, err := s.p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err) - p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain|flagPushDownAgg, p.(LogicalPlan)) - require.NoError(t, err) - testdata.OnRecord(func() { - output[ith] = ToString(p) - }) - require.Equal(t, output[ith], ToString(p), fmt.Sprintf("for %s %d", tt, ith)) - } -} - -func TestColumnPruning(t *testing.T) { - var ( - input []string - output []map[int][]string - ) - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - - s := createPlannerSuite() - ctx := context.Background() - for i, tt := range input { - comment := fmt.Sprintf("case:%v sql:\"%s\"", i, tt) - stmt, err := s.p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err) - lp, err := logicalOptimize(ctx, flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i] = make(map[int][]string) - }) - checkDataSourceCols(lp, t, output[i], comment) - } -} - -func TestSortByItemsPruning(t *testing.T) { - var ( - input []string - output [][]string - ) - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - testdata.OnRecord(func() { - output = make([][]string, len(input)) - }) - - s := createPlannerSuite() - ctx := context.Background() - for i, tt := range input { - comment := fmt.Sprintf("for %s", tt) - stmt, err := s.p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err) - lp, err := logicalOptimize(ctx, flagEliminateProjection|flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) - require.NoError(t, err) - checkOrderByItems(lp, t, &output[i], comment) - } -} - -func TestProjectionEliminator(t *testing.T) { - tests := []struct { - sql string - best string - }{ - { - sql: "select 1+num from (select 1+a as num from t) t1;", - best: "DataScan(t)->Projection", - }, { - sql: "select count(*) from t where a in (select b from t2 where a is null);", - best: "Join{DataScan(t)->Dual->Aggr(firstrow(test.t2.b))}(test.t.a,test.t2.b)->Aggr(count(1))->Projection", - }, - } - - s := createPlannerSuite() - ctx := context.Background() - for ith, tt := range tests { - comment := fmt.Sprintf("for %s", tt.sql) - stmt, err := s.p.ParseOneStmt(tt.sql, "", "") - require.NoError(t, err, comment) - - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err) - p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagPrunColumns|flagPrunColumnsAgain|flagEliminateProjection, p.(LogicalPlan)) - require.NoError(t, err) - require.Equal(t, tt.best, ToString(p), fmt.Sprintf("for %s %d", tt.sql, ith)) - } -} - -func TestCS3389(t *testing.T) { - s := createPlannerSuite() - ctx := context.Background() - stmt, err := s.p.ParseOneStmt("select count(*) from t where a in (select b from t2 where a is null);", "", "") - require.NoError(t, err) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err) - p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagPrunColumns|flagPrunColumnsAgain|flagEliminateProjection|flagJoinReOrder, p.(LogicalPlan)) - require.NoError(t, err) - - // Assert that all Projection is not empty and there is no Projection between Aggregation and Join. - proj, isProj := p.(*LogicalProjection) - require.True(t, isProj) - require.True(t, len(proj.Exprs) > 0) - child := proj.Children()[0] - agg, isAgg := child.(*LogicalAggregation) - require.True(t, isAgg) - child = agg.Children()[0] - _, isJoin := child.(*LogicalJoin) - require.True(t, isJoin) -} - -func TestAllocID(t *testing.T) { - ctx := MockContext() - pA := DataSource{}.Init(ctx, 0) - pB := DataSource{}.Init(ctx, 0) - require.Equal(t, pB.id, pA.id+1) -} - -func checkDataSourceCols(p LogicalPlan, t *testing.T, ans map[int][]string, comment string) { - switch v := p.(type) { - case *DataSource, *LogicalUnionAll, *LogicalLimit: - testdata.OnRecord(func() { - ans[p.ID()] = make([]string, p.Schema().Len()) - }) - colList, ok := ans[p.ID()] - require.True(t, ok, fmt.Sprintf("For %s %T ID %d Not found", comment, v, p.ID())) - require.Equal(t, len(colList), len(p.Schema().Columns), comment) - for i, col := range p.Schema().Columns { - testdata.OnRecord(func() { - colList[i] = col.String() - }) - require.Equal(t, colList[i], col.String(), comment) - } - } - for _, child := range p.Children() { - checkDataSourceCols(child, t, ans, comment) - } -} - -func checkOrderByItems(p LogicalPlan, t *testing.T, colList *[]string, comment string) { - switch p := p.(type) { - case *LogicalSort: - testdata.OnRecord(func() { - *colList = make([]string, len(p.ByItems)) - }) - for i, col := range p.ByItems { - testdata.OnRecord(func() { - (*colList)[i] = col.String() - }) - s := col.String() - require.Equal(t, (*colList)[i], s, comment) - } - } - children := p.Children() - require.LessOrEqual(t, len(children), 1, fmt.Sprintf("For %v Expected <= 1 Child", comment)) - for _, child := range children { - checkOrderByItems(child, t, colList, comment) - } -} - -func TestValidate(t *testing.T) { - tests := []struct { - sql string - err *terror.Error - }{ - { - sql: "select date_format((1,2), '%H');", - err: expression.ErrOperandColumns, - }, - { - sql: "select cast((1,2) as date)", - err: expression.ErrOperandColumns, - }, - { - sql: "select (1,2) between (3,4) and (5,6)", - err: expression.ErrOperandColumns, - }, - { - sql: "select (1,2) rlike '1'", - err: expression.ErrOperandColumns, - }, - { - sql: "select (1,2) like '1'", - err: expression.ErrOperandColumns, - }, - { - sql: "select case(1,2) when(1,2) then true end", - err: expression.ErrOperandColumns, - }, - { - sql: "select (1,2) in ((3,4),(5,6))", - err: nil, - }, - { - sql: "select row(1,(2,3)) in (select a,b from t)", - err: expression.ErrOperandColumns, - }, - { - sql: "select row(1,2) in (select a,b from t)", - err: nil, - }, - { - sql: "select (1,2) in ((3,4),5)", - err: expression.ErrOperandColumns, - }, - { - sql: "select (1,2) is true", - err: expression.ErrOperandColumns, - }, - { - sql: "select (1,2) is null", - err: expression.ErrOperandColumns, - }, - { - sql: "select (+(1,2))=(1,2)", - err: nil, - }, - { - sql: "select (-(1,2))=(1,2)", - err: expression.ErrOperandColumns, - }, - { - sql: "select (1,2)||(1,2)", - err: expression.ErrOperandColumns, - }, - { - sql: "select (1,2) < (3,4)", - err: nil, - }, - { - sql: "select (1,2) < 3", - err: expression.ErrOperandColumns, - }, - { - sql: "select 1, * from t", - err: ErrInvalidWildCard, - }, - { - sql: "select *, 1 from t", - err: nil, - }, - { - sql: "select 1, t.* from t", - err: nil, - }, - { - sql: "select 1 from t t1, t t2 where t1.a > all((select a) union (select a))", - err: ErrAmbiguous, - }, - { - sql: "insert into t set a = 1, b = a + 1", - err: nil, - }, - { - sql: "insert into t set a = 1, b = values(a) + 1", - err: nil, - }, - { - sql: "select a, b, c from t order by 0", - err: ErrUnknownColumn, - }, - { - sql: "select a, b, c from t order by 4", - err: ErrUnknownColumn, - }, - { - sql: "select a as c1, b as c1 from t order by c1", - err: ErrAmbiguous, - }, - { - sql: "(select a as b, b from t) union (select a, b from t) order by b", - err: ErrAmbiguous, - }, - { - sql: "(select a as b, b from t) union (select a, b from t) order by a", - err: ErrUnknownColumn, - }, - { - sql: "select * from t t1 use index(x)", - err: ErrKeyDoesNotExist, - }, - { - sql: "select a from t having c2", - err: ErrUnknownColumn, - }, - { - sql: "select a from t group by c2 + 1 having c2", - err: ErrUnknownColumn, - }, - { - sql: "select a as b, b from t having b", - err: ErrAmbiguous, - }, - { - sql: "select a + 1 from t having a", - err: ErrUnknownColumn, - }, - { // issue (#20509) - sql: "select * from t left join t2 on t.a=t2.a having not (t.a <=> t2.a)", - err: nil, - }, - { - sql: "select a from t having sum(avg(a))", - err: ErrInvalidGroupFuncUse, - }, - { - sql: "select concat(c_str, d_str) from t group by `concat(c_str, d_str)`", - err: nil, - }, - { - sql: "select concat(c_str, d_str) from t group by `concat(c_str,d_str)`", - err: ErrUnknownColumn, - }, - { - sql: "select a from t b having b.a", - err: nil, - }, - { - sql: "select b.a from t b having b.a", - err: nil, - }, - { - sql: "select b.a from t b having a", - err: nil, - }, - { - sql: "select a+1 from t having t.a", - err: ErrUnknownColumn, - }, - { - sql: "update T_StateNoneColumn set c = 1 where a = 1", - err: ErrUnknownColumn, - }, - } - - s := createPlannerSuite() - ctx := context.Background() - for _, tt := range tests { - sql := tt.sql - comment := fmt.Sprintf("for %s", sql) - stmt, err := s.p.ParseOneStmt(sql, "", "") - require.NoError(t, err, comment) - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err) - _, _, err = BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - if tt.err == nil { - require.NoError(t, err, comment) - } else { - require.True(t, tt.err.Equal(err), comment) - } - } -} - -func checkUniqueKeys(p LogicalPlan, t *testing.T, ans map[int][][]string, sql string) { - testdata.OnRecord(func() { - ans[p.ID()] = make([][]string, len(p.Schema().Keys)) - }) - keyList, ok := ans[p.ID()] - require.True(t, ok, fmt.Sprintf("for %s, %v not found", sql, p.ID())) - require.Equal(t, len(keyList), len(p.Schema().Keys), fmt.Sprintf("for %s, %v, the number of key doesn't match, the schema is %s", sql, p.ID(), p.Schema())) - for i := range keyList { - testdata.OnRecord(func() { - keyList[i] = make([]string, len(p.Schema().Keys[i])) - }) - require.Equal(t, len(keyList[i]), len(p.Schema().Keys[i]), fmt.Sprintf("for %s, %v %v, the number of column doesn't match", sql, p.ID(), keyList[i])) - for j := range keyList[i] { - testdata.OnRecord(func() { - keyList[i][j] = p.Schema().Keys[i][j].String() - }) - require.Equal(t, keyList[i][j], p.Schema().Keys[i][j].String(), fmt.Sprintf("for %s, %v %v, column dosen't match", sql, p.ID(), keyList[i])) - } - } - testdata.OnRecord(func() { - ans[p.ID()] = keyList - }) - for _, child := range p.Children() { - checkUniqueKeys(child, t, ans, sql) - } -} - -func TestUniqueKeyInfo(t *testing.T) { - var input []string - var output []map[int][][]string - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - testdata.OnRecord(func() { - output = make([]map[int][][]string, len(input)) - }) - - s := createPlannerSuite() - ctx := context.Background() - for ith, tt := range input { - comment := fmt.Sprintf("for %s %d", tt, ith) - stmt, err := s.p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err) - lp, err := logicalOptimize(context.TODO(), flagPredicatePushDown|flagPrunColumns|flagBuildKeyInfo, p.(LogicalPlan)) - require.NoError(t, err) - testdata.OnRecord(func() { - output[ith] = make(map[int][][]string) - }) - checkUniqueKeys(lp, t, output[ith], tt) - } -} - -func TestAggPrune(t *testing.T) { - var input, output []string - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - - s := createPlannerSuite() - ctx := context.Background() - for i, tt := range input { - comment := fmt.Sprintf("for %s", tt) - stmt, err := s.p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - domain.GetDomain(s.ctx).MockInfoCacheAndLoadInfoSchema(s.is) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err) - - p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain|flagBuildKeyInfo|flagEliminateAgg|flagEliminateProjection, p.(LogicalPlan)) - require.NoError(t, err) - planString := ToString(p) - testdata.OnRecord(func() { - output[i] = planString - }) - require.Equal(t, output[i], planString, comment) - } -} - -func TestVisitInfo(t *testing.T) { - variable.EnableMDL.Store(false) - tests := []struct { - sql string - ans []visitInfo - }{ - { - sql: "insert into t (a) values (1)", - ans: []visitInfo{ - {mysql.InsertPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "delete from t where a = 1", - ans: []visitInfo{ - {mysql.DeletePriv, "test", "t", "", nil, false, "", false}, - {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "delete from t order by a", - ans: []visitInfo{ - {mysql.DeletePriv, "test", "t", "", nil, false, "", false}, - {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "delete from t", - ans: []visitInfo{ - {mysql.DeletePriv, "test", "t", "", nil, false, "", false}, - }, - }, - /* Not currently supported. See https://github.com/pingcap/tidb/issues/23644 - { - sql: "delete from t where 1=1", - ans: []visitInfo{ - {mysql.DeletePriv, "test", "t", "", nil, false, "", false}, - }, - }, - */ - { - sql: "delete from a1 using t as a1 inner join t as a2 where a1.a = a2.a", - ans: []visitInfo{ - {mysql.DeletePriv, "test", "t", "", nil, false, "", false}, - {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "update t set a = 7 where a = 1", - ans: []visitInfo{ - {mysql.UpdatePriv, "test", "t", "", nil, false, "", false}, - {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "update t, (select * from t) a1 set t.a = a1.a;", - ans: []visitInfo{ - {mysql.UpdatePriv, "test", "t", "", nil, false, "", false}, - {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "update t a1 set a1.a = a1.a + 1", - ans: []visitInfo{ - {mysql.UpdatePriv, "test", "t", "", nil, false, "", false}, - {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "select a, sum(e) from t group by a", - ans: []visitInfo{ - {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "truncate table t", - ans: []visitInfo{ - {mysql.DropPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "drop table t", - ans: []visitInfo{ - {mysql.DropPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "create table t (a int)", - ans: []visitInfo{ - {mysql.CreatePriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "create table t1 like t", - ans: []visitInfo{ - {mysql.CreatePriv, "test", "t1", "", nil, false, "", false}, - {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "create database test", - ans: []visitInfo{ - {mysql.CreatePriv, "test", "", "", nil, false, "", false}, - }, - }, - { - sql: "drop database test", - ans: []visitInfo{ - {mysql.DropPriv, "test", "", "", nil, false, "", false}, - }, - }, - { - sql: "create index t_1 on t (a)", - ans: []visitInfo{ - {mysql.IndexPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "drop index e on t", - ans: []visitInfo{ - {mysql.IndexPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: `grant all privileges on test.* to 'test'@'%'`, - ans: []visitInfo{ - {mysql.SelectPriv, "test", "", "", nil, false, "", false}, - {mysql.InsertPriv, "test", "", "", nil, false, "", false}, - {mysql.UpdatePriv, "test", "", "", nil, false, "", false}, - {mysql.DeletePriv, "test", "", "", nil, false, "", false}, - {mysql.CreatePriv, "test", "", "", nil, false, "", false}, - {mysql.DropPriv, "test", "", "", nil, false, "", false}, - {mysql.GrantPriv, "test", "", "", nil, false, "", false}, - {mysql.ReferencesPriv, "test", "", "", nil, false, "", false}, - {mysql.LockTablesPriv, "test", "", "", nil, false, "", false}, - {mysql.CreateTMPTablePriv, "test", "", "", nil, false, "", false}, - {mysql.EventPriv, "test", "", "", nil, false, "", false}, - {mysql.CreateRoutinePriv, "test", "", "", nil, false, "", false}, - {mysql.AlterRoutinePriv, "test", "", "", nil, false, "", false}, - {mysql.AlterPriv, "test", "", "", nil, false, "", false}, - {mysql.ExecutePriv, "test", "", "", nil, false, "", false}, - {mysql.IndexPriv, "test", "", "", nil, false, "", false}, - {mysql.CreateViewPriv, "test", "", "", nil, false, "", false}, - {mysql.ShowViewPriv, "test", "", "", nil, false, "", false}, - {mysql.TriggerPriv, "test", "", "", nil, false, "", false}, - }, - }, - { - sql: `grant all privileges on *.* to 'test'@'%'`, - ans: []visitInfo{ - {mysql.SelectPriv, "", "", "", nil, false, "", false}, - {mysql.InsertPriv, "", "", "", nil, false, "", false}, - {mysql.UpdatePriv, "", "", "", nil, false, "", false}, - {mysql.DeletePriv, "", "", "", nil, false, "", false}, - {mysql.CreatePriv, "", "", "", nil, false, "", false}, - {mysql.DropPriv, "", "", "", nil, false, "", false}, - {mysql.ProcessPriv, "", "", "", nil, false, "", false}, - {mysql.ReferencesPriv, "", "", "", nil, false, "", false}, - {mysql.AlterPriv, "", "", "", nil, false, "", false}, - {mysql.ShowDBPriv, "", "", "", nil, false, "", false}, - {mysql.SuperPriv, "", "", "", nil, false, "", false}, - {mysql.ExecutePriv, "", "", "", nil, false, "", false}, - {mysql.IndexPriv, "", "", "", nil, false, "", false}, - {mysql.CreateUserPriv, "", "", "", nil, false, "", false}, - {mysql.CreateTablespacePriv, "", "", "", nil, false, "", false}, - {mysql.TriggerPriv, "", "", "", nil, false, "", false}, - {mysql.CreateViewPriv, "", "", "", nil, false, "", false}, - {mysql.ShowViewPriv, "", "", "", nil, false, "", false}, - {mysql.CreateRolePriv, "", "", "", nil, false, "", false}, - {mysql.DropRolePriv, "", "", "", nil, false, "", false}, - {mysql.CreateTMPTablePriv, "", "", "", nil, false, "", false}, - {mysql.LockTablesPriv, "", "", "", nil, false, "", false}, - {mysql.CreateRoutinePriv, "", "", "", nil, false, "", false}, - {mysql.AlterRoutinePriv, "", "", "", nil, false, "", false}, - {mysql.EventPriv, "", "", "", nil, false, "", false}, - {mysql.ShutdownPriv, "", "", "", nil, false, "", false}, - {mysql.ReloadPriv, "", "", "", nil, false, "", false}, - {mysql.FilePriv, "", "", "", nil, false, "", false}, - {mysql.ConfigPriv, "", "", "", nil, false, "", false}, - {mysql.ReplicationClientPriv, "", "", "", nil, false, "", false}, - {mysql.ReplicationSlavePriv, "", "", "", nil, false, "", false}, - {mysql.GrantPriv, "", "", "", nil, false, "", false}, - }, - }, - { - sql: `grant select on test.ttt to 'test'@'%'`, - ans: []visitInfo{ - {mysql.SelectPriv, "test", "ttt", "", nil, false, "", false}, - {mysql.GrantPriv, "test", "ttt", "", nil, false, "", false}, - }, - }, - { - sql: `grant select on ttt to 'test'@'%'`, - ans: []visitInfo{ - {mysql.SelectPriv, "test", "ttt", "", nil, false, "", false}, - {mysql.GrantPriv, "test", "ttt", "", nil, false, "", false}, - }, - }, - { - sql: `revoke all privileges on test.* from 'test'@'%'`, - ans: []visitInfo{ - {mysql.SelectPriv, "test", "", "", nil, false, "", false}, - {mysql.InsertPriv, "test", "", "", nil, false, "", false}, - {mysql.UpdatePriv, "test", "", "", nil, false, "", false}, - {mysql.DeletePriv, "test", "", "", nil, false, "", false}, - {mysql.CreatePriv, "test", "", "", nil, false, "", false}, - {mysql.DropPriv, "test", "", "", nil, false, "", false}, - {mysql.GrantPriv, "test", "", "", nil, false, "", false}, - {mysql.ReferencesPriv, "test", "", "", nil, false, "", false}, - {mysql.LockTablesPriv, "test", "", "", nil, false, "", false}, - {mysql.CreateTMPTablePriv, "test", "", "", nil, false, "", false}, - {mysql.EventPriv, "test", "", "", nil, false, "", false}, - {mysql.CreateRoutinePriv, "test", "", "", nil, false, "", false}, - {mysql.AlterRoutinePriv, "test", "", "", nil, false, "", false}, - {mysql.AlterPriv, "test", "", "", nil, false, "", false}, - {mysql.ExecutePriv, "test", "", "", nil, false, "", false}, - {mysql.IndexPriv, "test", "", "", nil, false, "", false}, - {mysql.CreateViewPriv, "test", "", "", nil, false, "", false}, - {mysql.ShowViewPriv, "test", "", "", nil, false, "", false}, - {mysql.TriggerPriv, "test", "", "", nil, false, "", false}, - }, - }, - { - sql: `revoke connection_admin on *.* from u1`, - ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", nil, false, "CONNECTION_ADMIN", true}, - }, - }, - { - sql: `revoke connection_admin, select on *.* from u1`, - ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", nil, false, "CONNECTION_ADMIN", true}, - {mysql.SelectPriv, "", "", "", nil, false, "", false}, - {mysql.GrantPriv, "", "", "", nil, false, "", false}, - }, - }, - { - sql: `revoke all privileges on *.* FROM u1`, - ans: []visitInfo{ - {mysql.SelectPriv, "", "", "", nil, false, "", false}, - {mysql.InsertPriv, "", "", "", nil, false, "", false}, - {mysql.UpdatePriv, "", "", "", nil, false, "", false}, - {mysql.DeletePriv, "", "", "", nil, false, "", false}, - {mysql.CreatePriv, "", "", "", nil, false, "", false}, - {mysql.DropPriv, "", "", "", nil, false, "", false}, - {mysql.ProcessPriv, "", "", "", nil, false, "", false}, - {mysql.ReferencesPriv, "", "", "", nil, false, "", false}, - {mysql.AlterPriv, "", "", "", nil, false, "", false}, - {mysql.ShowDBPriv, "", "", "", nil, false, "", false}, - {mysql.SuperPriv, "", "", "", nil, false, "", false}, - {mysql.ExecutePriv, "", "", "", nil, false, "", false}, - {mysql.IndexPriv, "", "", "", nil, false, "", false}, - {mysql.CreateUserPriv, "", "", "", nil, false, "", false}, - {mysql.CreateTablespacePriv, "", "", "", nil, false, "", false}, - {mysql.TriggerPriv, "", "", "", nil, false, "", false}, - {mysql.CreateViewPriv, "", "", "", nil, false, "", false}, - {mysql.ShowViewPriv, "", "", "", nil, false, "", false}, - {mysql.CreateRolePriv, "", "", "", nil, false, "", false}, - {mysql.DropRolePriv, "", "", "", nil, false, "", false}, - {mysql.CreateTMPTablePriv, "", "", "", nil, false, "", false}, - {mysql.LockTablesPriv, "", "", "", nil, false, "", false}, - {mysql.CreateRoutinePriv, "", "", "", nil, false, "", false}, - {mysql.AlterRoutinePriv, "", "", "", nil, false, "", false}, - {mysql.EventPriv, "", "", "", nil, false, "", false}, - {mysql.ShutdownPriv, "", "", "", nil, false, "", false}, - {mysql.ReloadPriv, "", "", "", nil, false, "", false}, - {mysql.FilePriv, "", "", "", nil, false, "", false}, - {mysql.ConfigPriv, "", "", "", nil, false, "", false}, - {mysql.ReplicationClientPriv, "", "", "", nil, false, "", false}, - {mysql.ReplicationSlavePriv, "", "", "", nil, false, "", false}, - {mysql.GrantPriv, "", "", "", nil, false, "", false}, - }, - }, - { - sql: `set password for 'root'@'%' = 'xxxxx'`, - ans: []visitInfo{}, - }, - { - sql: `show create table test.ttt`, - ans: []visitInfo{ - {mysql.AllPrivMask, "test", "ttt", "", nil, false, "", false}, - }, - }, - { - sql: "alter table t add column a int(4)", - ans: []visitInfo{ - {mysql.AlterPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "rename table t_old to t_new", - ans: []visitInfo{ - {mysql.AlterPriv, "test", "t_old", "", nil, false, "", false}, - {mysql.DropPriv, "test", "t_old", "", nil, false, "", false}, - {mysql.CreatePriv, "test", "t_new", "", nil, false, "", false}, - {mysql.InsertPriv, "test", "t_new", "", nil, false, "", false}, - }, - }, - { - sql: "alter table t_old rename to t_new", - ans: []visitInfo{ - {mysql.AlterPriv, "test", "t_old", "", nil, false, "", false}, - {mysql.DropPriv, "test", "t_old", "", nil, false, "", false}, - {mysql.CreatePriv, "test", "t_new", "", nil, false, "", false}, - {mysql.InsertPriv, "test", "t_new", "", nil, false, "", false}, - }, - }, - { - sql: "alter table t drop partition p0;", - ans: []visitInfo{ - {mysql.AlterPriv, "test", "t", "", nil, false, "", false}, - {mysql.DropPriv, "test", "t", "", nil, false, "", false}, - }, - }, - { - sql: "flush privileges", - ans: []visitInfo{ - {mysql.ReloadPriv, "", "", "", ErrSpecificAccessDenied, false, "", false}, - }, - }, - { - sql: "SET GLOBAL wait_timeout=12345", - ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "SYSTEM_VARIABLES_ADMIN", false}, - }, - }, - { - sql: "create placement policy x LEARNERS=1", - ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "PLACEMENT_ADMIN", false}, - }, - }, - { - sql: "drop placement policy if exists x", - ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "PLACEMENT_ADMIN", false}, - }, - }, - { - sql: "BACKUP DATABASE test TO 'local:///tmp/a'", - ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "BACKUP_ADMIN", false}, - }, - }, - { - sql: "RESTORE DATABASE test FROM 'local:///tmp/a'", - ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "RESTORE_ADMIN", false}, - }, - }, - { - sql: "SHOW BACKUPS", - ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "BACKUP_ADMIN", false}, - }, - }, - { - sql: "SHOW RESTORES", - ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "RESTORE_ADMIN", false}, - }, - }, - { - sql: "GRANT rolename TO user1", - ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "ROLE_ADMIN", false}, - }, - }, - { - sql: "REVOKE rolename FROM user1", - ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "ROLE_ADMIN", false}, - }, - }, - { - sql: "GRANT BACKUP_ADMIN ON *.* TO user1", - ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "BACKUP_ADMIN", true}, - }, - }, - { - sql: "GRANT BACKUP_ADMIN ON *.* TO user1 WITH GRANT OPTION", - ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "BACKUP_ADMIN", true}, - }, - }, - { - sql: "RENAME USER user1 to user1_tmp", - ans: []visitInfo{ - {mysql.CreateUserPriv, "", "", "", ErrSpecificAccessDenied, false, "", false}, - }, - }, - { - sql: "SHOW CONFIG", - ans: []visitInfo{ - {mysql.ConfigPriv, "", "", "", ErrSpecificAccessDenied, false, "", false}, - }, - }, - } - - s := createPlannerSuite() - for _, tt := range tests { - comment := fmt.Sprintf("for %s", tt.sql) - stmt, err := s.p.ParseOneStmt(tt.sql, "", "") - require.NoError(t, err, comment) - - // TODO: to fix, Table 'test.ttt' doesn't exist - _ = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - sctx := MockContext() - builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) - domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) - builder.ctx.GetSessionVars().SetHashJoinConcurrency(1) - _, err = builder.Build(context.TODO(), stmt) - require.NoError(t, err, comment) - - checkVisitInfo(t, builder.visitInfo, tt.ans, comment) - } -} - -type visitInfoArray []visitInfo - -func (v visitInfoArray) Len() int { - return len(v) -} - -func (v visitInfoArray) Less(i, j int) bool { - if v[i].privilege < v[j].privilege { - return true - } - if v[i].db < v[j].db { - return true - } - if v[i].table < v[j].table { - return true - } - if v[i].column < v[j].column { - return true - } - - return false -} - -func (v visitInfoArray) Swap(i, j int) { - v[i], v[j] = v[j], v[i] -} - -func unique(v []visitInfo) []visitInfo { - repeat := 0 - for i := 1; i < len(v); i++ { - if v[i] == v[i-1] { - repeat++ - } else { - v[i-repeat] = v[i] - } - } - return v[:len(v)-repeat] -} - -func checkVisitInfo(t *testing.T, v1, v2 []visitInfo, comment string) { - sort.Sort(visitInfoArray(v1)) - sort.Sort(visitInfoArray(v2)) - v1 = unique(v1) - v2 = unique(v2) - - require.Equal(t, len(v2), len(v1), comment) - for i := 0; i < len(v1); i++ { - // loose compare errors for code match - require.True(t, terror.ErrorEqual(v1[i].err, v2[i].err), fmt.Sprintf("err1 %v, err2 %v for %s", v1[i].err, v2[i].err, comment)) - // compare remainder - v1[i].err = v2[i].err - require.Equal(t, v2[i], v1[i], comment) - } -} - -func TestUnion(t *testing.T) { - var input []string - var output []struct { - Best string - Err bool - } - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - s := createPlannerSuite() - ctx := context.TODO() - for i, tt := range input { - comment := fmt.Sprintf("case:%v sql:%s", i, tt) - stmt, err := s.p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err) - sctx := MockContext() - builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) - domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) - plan, err := builder.Build(ctx, stmt) - testdata.OnRecord(func() { - output[i].Err = err != nil - }) - if output[i].Err { - require.Error(t, err) - continue - } - require.NoError(t, err, comment) - p := plan.(LogicalPlan) - p, err = logicalOptimize(ctx, builder.optFlag, p) - testdata.OnRecord(func() { - output[i].Best = ToString(p) - }) - require.NoError(t, err) - require.Equal(t, output[i].Best, ToString(p), comment) - } -} - -func TestTopNPushDown(t *testing.T) { - var input, output []string - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - s := createPlannerSuite() - ctx := context.TODO() - for i, tt := range input { - comment := fmt.Sprintf("case:%v sql:%s", i, tt) - stmt, err := s.p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err) - sctx := MockContext() - builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) - domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) - p, err := builder.Build(ctx, stmt) - require.NoError(t, err) - p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i] = ToString(p) - }) - require.Equal(t, output[i], ToString(p), comment) - } -} - -func TestNameResolver(t *testing.T) { - tests := []struct { - sql string - err string - }{ - {"select a from t", ""}, - {"select c3 from t", "[planner:1054]Unknown column 'c3' in 'field list'"}, - {"select c1 from t4", "[schema:1146]Table 'test.t4' doesn't exist"}, - {"select * from t", ""}, - {"select t.* from t", ""}, - {"select t2.* from t", "[planner:1051]Unknown table 't2'"}, - {"select b as a, c as a from t group by a", "[planner:1052]Column 'a' in group statement is ambiguous"}, - {"select 1 as a, b as a, c as a from t group by a", ""}, - {"select a, b as a from t group by a+1", ""}, - {"select c, a as c from t order by c+1", ""}, - {"select * from t as t1, t as t2 join t as t3 on t2.a = t3.a", ""}, - {"select * from t as t1, t as t2 join t as t3 on t1.c1 = t2.a", "[planner:1054]Unknown column 't1.c1' in 'on clause'"}, - {"select a from t group by a having a = 3", ""}, - {"select a from t group by a having c2 = 3", "[planner:1054]Unknown column 'c2' in 'having clause'"}, - {"select a from t where exists (select b)", ""}, - {"select cnt from (select count(a) as cnt from t group by b) as t2 group by cnt", ""}, - {"select a from t where t11.a < t.a", "[planner:1054]Unknown column 't11.a' in 'where clause'"}, - {"select a from t having t11.c1 < t.a", "[planner:1054]Unknown column 't11.c1' in 'having clause'"}, - {"select a from t where t.a < t.a order by t11.c1", "[planner:1054]Unknown column 't11.c1' in 'order clause'"}, - {"select a from t group by t11.c1", "[planner:1054]Unknown column 't11.c1' in 'group statement'"}, - {"delete a from (select * from t ) as a, t", "[planner:1288]The target table a of the DELETE is not updatable"}, - {"delete b from (select * from t ) as a, t", "[planner:1109]Unknown table 'b' in MULTI DELETE"}, - {"select '' as fakeCol from t group by values(fakeCol)", "[planner:1054]Unknown column '' in 'VALUES() function'"}, - {"update t, (select * from t) as b set b.a = t.a", "[planner:1288]The target table b of the UPDATE is not updatable"}, - {"select row_number() over () from t group by 1", "[planner:1056]Can't group on 'row_number() over ()'"}, - {"select row_number() over () as x from t group by 1", "[planner:1056]Can't group on 'x'"}, - {"select sum(a) as x from t group by 1", "[planner:1056]Can't group on 'x'"}, - } - - s := createPlannerSuite() - ctx := context.Background() - for _, test := range tests { - comment := fmt.Sprintf("for %s", test.sql) - stmt, err := s.p.ParseOneStmt(test.sql, "", "") - require.NoError(t, err, comment) - s.ctx.GetSessionVars().SetHashJoinConcurrency(1) - - _, _, err = BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - if test.err == "" { - require.NoError(t, err) - } else { - require.EqualError(t, err, test.err) - } - } -} - -func TestOuterJoinEliminator(t *testing.T) { - var input, output []string - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - - s := createPlannerSuite() - ctx := context.TODO() - for i, tt := range input { - comment := fmt.Sprintf("case:%v sql:%s", i, tt) - stmt, err := s.p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err) - sctx := MockContext() - builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) - domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) - p, err := builder.Build(ctx, stmt) - require.NoError(t, err) - p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) - require.NoError(t, err) - planString := ToString(p) - testdata.OnRecord(func() { - output[i] = planString - }) - require.Equal(t, output[i], planString, comment) - } -} - -func TestSelectView(t *testing.T) { - tests := []struct { - sql string - best string - }{ - { - sql: "select * from v", - best: "DataScan(t)->Projection", - }, - { - sql: "select v.b, v.c, v.d from v", - best: "DataScan(t)->Projection", - }, - } - s := createPlannerSuite() - ctx := context.TODO() - for i, tt := range tests { - comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) - stmt, err := s.p.ParseOneStmt(tt.sql, "", "") - require.NoError(t, err, comment) - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err) - builder, _ := NewPlanBuilder().Init(MockContext(), s.is, &hint.BlockHintProcessor{}) - p, err := builder.Build(ctx, stmt) - require.NoError(t, err) - p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) - require.NoError(t, err) - require.Equal(t, tt.best, ToString(p), comment) - } -} - -type plannerSuiteWithOptimizeVars struct { - *plannerSuite - optimizeVars map[string]string -} - -func TestWindowFunction(t *testing.T) { - s := new(plannerSuiteWithOptimizeVars) - s.plannerSuite = createPlannerSuite() - - s.optimizeVars = map[string]string{ - variable.TiDBWindowConcurrency: "1", - variable.TiDBCostModelVersion: "1", - } - defer func() { - s.optimizeVars = nil - }() - var input, output []string - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - s.doTestWindowFunction(t, input, output) -} - -func TestWindowParallelFunction(t *testing.T) { - s := new(plannerSuiteWithOptimizeVars) - s.plannerSuite = createPlannerSuite() - - s.optimizeVars = map[string]string{ - variable.TiDBWindowConcurrency: "4", - variable.TiDBCostModelVersion: "1", - } - defer func() { - s.optimizeVars = nil - }() - var input, output []string - planSuiteUnexportedData.LoadTestCases(t, &input, &output) - s.doTestWindowFunction(t, input, output) -} - -func (s *plannerSuiteWithOptimizeVars) doTestWindowFunction(t *testing.T, input, output []string) { - ctx := context.TODO() - for i, tt := range input { - comment := fmt.Sprintf("case:%v sql:%s", i, tt) - p, stmt, err := s.optimize(ctx, tt) - if err != nil { - testdata.OnRecord(func() { - output[i] = err.Error() - }) - require.EqualError(t, err, output[i], comment) - continue - } - testdata.OnRecord(func() { - output[i] = ToString(p) - }) - require.Equal(t, output[i], ToString(p), comment) - - var sb strings.Builder - // After restore, the result should be the same. - err = stmt.Restore(format.NewRestoreCtx(format.DefaultRestoreFlags, &sb)) - require.NoError(t, err) - p, _, err = s.optimize(ctx, sb.String()) - if err != nil { - require.EqualError(t, err, output[i], comment) - continue - } - require.Equal(t, output[i], ToString(p), comment) - } -} - -func (s *plannerSuiteWithOptimizeVars) optimize(ctx context.Context, sql string) (PhysicalPlan, ast.Node, error) { - stmt, err := s.p.ParseOneStmt(sql, "", "") - if err != nil { - return nil, nil, err - } - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - if err != nil { - return nil, nil, err - } - - sctx := MockContext() - for k, v := range s.optimizeVars { - if err = sctx.GetSessionVars().SetSystemVar(k, v); err != nil { - return nil, nil, err - } - } - builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) - domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) - p, err := builder.Build(ctx, stmt) - if err != nil { - return nil, nil, err - } - p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) - if err != nil { - return nil, nil, err - } - p, _, err = physicalOptimize(p.(LogicalPlan), &PlanCounterDisabled) - return p.(PhysicalPlan), stmt, err -} - -func byItemsToProperty(byItems []*util.ByItems) *property.PhysicalProperty { - pp := &property.PhysicalProperty{} - for _, item := range byItems { - pp.SortItems = append(pp.SortItems, property.SortItem{Col: item.Expr.(*expression.Column), Desc: item.Desc}) - } - return pp -} - -func pathsName(paths []*candidatePath) string { - var names []string - for _, path := range paths { - if path.path.IsTablePath() { - names = append(names, "PRIMARY_KEY") - } else { - names = append(names, path.path.Index.Name.O) - } - } - return strings.Join(names, ",") -} - -func TestSkylinePruning(t *testing.T) { - tests := []struct { - sql string - result string - }{ - { - sql: "select * from t", - result: "PRIMARY_KEY", - }, - { - sql: "select * from t order by f", - result: "PRIMARY_KEY,f,f_g", - }, - { - sql: "select * from t where a > 1", - result: "PRIMARY_KEY", - }, - { - sql: "select * from t where a > 1 order by f", - result: "PRIMARY_KEY,f,f_g", - }, - { - sql: "select * from t where f > 1", - result: "PRIMARY_KEY,f,f_g", - }, - { - sql: "select f from t where f > 1", - result: "f,f_g", - }, - { - sql: "select f from t where f > 1 order by a", - result: "PRIMARY_KEY,f,f_g", - }, - { - sql: "select * from t where f > 1 and g > 1", - result: "PRIMARY_KEY,g,f_g", - }, - { - sql: "select count(1) from t", - result: "PRIMARY_KEY,c_d_e,f,g,f_g,c_d_e_str,e_d_c_str_prefix", - }, - { - sql: "select * from t where f > 3 and g = 5", - result: "PRIMARY_KEY,g,f_g", - }, - { - sql: "select * from t where g = 5 order by f", - result: "PRIMARY_KEY,g,f_g", - }, - { - sql: "select * from t where d = 3 order by c, e", - result: "PRIMARY_KEY,c_d_e", - }, - { - sql: "select * from t where d = 1 and f > 1 and g > 1 order by c, e", - result: "PRIMARY_KEY,c_d_e,g,f_g", - }, - } - s := createPlannerSuite() - ctx := context.TODO() - for i, tt := range tests { - comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) - stmt, err := s.p.ParseOneStmt(tt.sql, "", "") - require.NoError(t, err, comment) - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err) - sctx := MockContext() - builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) - domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) - p, err := builder.Build(ctx, stmt) - if err != nil { - require.EqualError(t, err, tt.result, comment) - continue - } - require.NoError(t, err, comment) - p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) - require.NoError(t, err, comment) - lp := p.(LogicalPlan) - _, err = lp.recursiveDeriveStats(nil) - require.NoError(t, err, comment) - var ds *DataSource - var byItems []*util.ByItems - for ds == nil { - switch v := lp.(type) { - case *DataSource: - ds = v - case *LogicalSort: - byItems = v.ByItems - lp = lp.Children()[0] - case *LogicalProjection: - newItems := make([]*util.ByItems, 0, len(byItems)) - for _, col := range byItems { - idx := v.schema.ColumnIndex(col.Expr.(*expression.Column)) - switch expr := v.Exprs[idx].(type) { - case *expression.Column: - newItems = append(newItems, &util.ByItems{Expr: expr, Desc: col.Desc}) - } - } - byItems = newItems - lp = lp.Children()[0] - default: - lp = lp.Children()[0] - } - } - paths := ds.skylinePruning(byItemsToProperty(byItems)) - require.Equal(t, tt.result, pathsName(paths), comment) - } -} - -func TestFastPlanContextTables(t *testing.T) { - tests := []struct { - sql string - fastPlan bool - }{ - { - "select * from t where a=1", - true, - }, - { - - "update t set f=0 where a=43215", - true, - }, - { - "delete from t where a =43215", - true, - }, - { - "select * from t where a>1", - false, - }, - } - s := createPlannerSuite() - s.ctx.GetSessionVars().SnapshotInfoschema = s.is - for _, tt := range tests { - stmt, err := s.p.ParseOneStmt(tt.sql, "", "") - require.NoError(t, err) - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err) - s.ctx.GetSessionVars().StmtCtx.Tables = nil - p := TryFastPlan(s.ctx, stmt) - if tt.fastPlan { - require.NotNil(t, p) - require.Equal(t, 1, len(s.ctx.GetSessionVars().StmtCtx.Tables)) - require.Equal(t, "t", s.ctx.GetSessionVars().StmtCtx.Tables[0].Table) - require.Equal(t, "test", s.ctx.GetSessionVars().StmtCtx.Tables[0].DB) - } else { - require.Nil(t, p) - require.Equal(t, 0, len(s.ctx.GetSessionVars().StmtCtx.Tables)) - } - } -} - -func TestUpdateEQCond(t *testing.T) { - tests := []struct { - sql string - best string - }{ - { - sql: "select t1.a from t t1, t t2 where t1.a = t2.a+1", - best: "Join{DataScan(t1)->DataScan(t2)->Projection}(test.t.a,Column#25)->Projection->Projection", - }, - } - s := createPlannerSuite() - ctx := context.TODO() - for i, tt := range tests { - comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) - stmt, err := s.p.ParseOneStmt(tt.sql, "", "") - require.NoError(t, err, comment) - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err) - sctx := MockContext() - builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) - domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) - p, err := builder.Build(ctx, stmt) - require.NoError(t, err) - p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) - require.NoError(t, err) - require.Equal(t, tt.best, ToString(p), comment) - } -} - -func TestConflictedJoinTypeHints(t *testing.T) { - sql := "select /*+ INL_JOIN(t1) HASH_JOIN(t1) */ * from t t1, t t2 where t1.e = t2.e" - s := createPlannerSuite() - ctx := context.TODO() - stmt, err := s.p.ParseOneStmt(sql, "", "") - require.NoError(t, err) - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err) - sctx := MockContext() - builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) - domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) - p, err := builder.Build(ctx, stmt) - require.NoError(t, err) - p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) - require.NoError(t, err) - proj, ok := p.(*LogicalProjection) - require.True(t, ok) - join, ok := proj.Children()[0].(*LogicalJoin) - require.True(t, ok) - require.Nil(t, join.hintInfo) - require.Equal(t, uint(0), join.preferJoinType) -} - -func TestSimplyOuterJoinWithOnlyOuterExpr(t *testing.T) { - s := createPlannerSuite() - sql := "select * from t t1 right join t t0 ON TRUE where CONCAT_WS(t0.e=t0.e, 0, NULL) IS NULL" - ctx := context.TODO() - stmt, err := s.p.ParseOneStmt(sql, "", "") - require.NoError(t, err) - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err) - sctx := MockContext() - builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) - domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) - p, err := builder.Build(ctx, stmt) - require.NoError(t, err) - p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) - require.NoError(t, err) - proj, ok := p.(*LogicalProjection) - require.True(t, ok) - join, ok := proj.Children()[0].(*LogicalJoin) - require.True(t, ok) - // previous wrong JoinType is InnerJoin - require.Equal(t, RightOuterJoin, join.JoinType) -} - -func TestResolvingCorrelatedAggregate(t *testing.T) { - tests := []struct { - sql string - best string - }{ - { - sql: "select (select count(a)) from t", - best: "Apply{DataScan(t)->Aggr(count(test.t.a))->Dual->Projection->MaxOneRow}->Projection", - }, - { - sql: "select (select count(n.a) from t) from t n", - best: "Apply{DataScan(n)->Aggr(count(test.t.a))->DataScan(t)->Projection->MaxOneRow}->Projection", - }, - { - sql: "select (select sum(count(a))) from t", - best: "Apply{DataScan(t)->Aggr(count(test.t.a))->Dual->Aggr(sum(Column#13))->MaxOneRow}->Projection", - }, - { - sql: "select (select sum(count(n.a)) from t) from t n", - best: "Apply{DataScan(n)->Aggr(count(test.t.a))->DataScan(t)->Aggr(sum(Column#25))->MaxOneRow}->Projection", - }, - { - sql: "select (select cnt from (select count(a) as cnt) n) from t", - best: "Apply{DataScan(t)->Aggr(count(test.t.a))->Dual->Projection->MaxOneRow}->Projection", - }, - { - sql: "select sum(a), sum(a), count(a), (select count(a)) from t", - best: "Apply{DataScan(t)->Aggr(sum(test.t.a),count(test.t.a))->Dual->Projection->MaxOneRow}->Projection", - }, - } - - s := createPlannerSuite() - ctx := context.TODO() - for i, tt := range tests { - comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) - stmt, err := s.p.ParseOneStmt(tt.sql, "", "") - require.NoError(t, err, comment) - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err, comment) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err, comment) - p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagEliminateProjection|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) - require.NoError(t, err, comment) - require.Equal(t, tt.best, ToString(p), comment) - } -} - -func TestFastPathInvalidBatchPointGet(t *testing.T) { - // #22040 - tt := []struct { - sql string - fastPlan bool - }{ - { - // column count doesn't match, not use idx - sql: "select * from t where (a,b) in ((1,2),1)", - fastPlan: false, - }, - { - // column count doesn't match, not use idx - sql: "select * from t where (a,b) in (1,2)", - fastPlan: false, - }, - { - // column count doesn't match, use idx - sql: "select * from t where (f,g) in ((1,2),1)", - fastPlan: false, - }, - { - // column count doesn't match, use idx - sql: "select * from t where (f,g) in (1,2)", - fastPlan: false, - }, - } - s := createPlannerSuite() - for i, tc := range tt { - comment := fmt.Sprintf("case:%v sql:%s", i, tc.sql) - stmt, err := s.p.ParseOneStmt(tc.sql, "", "") - require.NoError(t, err, comment) - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err, comment) - plan := TryFastPlan(s.ctx, stmt) - if tc.fastPlan { - require.NotNil(t, plan) - } else { - require.Nil(t, plan) - } - } -} - -func TestTraceFastPlan(t *testing.T) { - s := createPlannerSuite() - s.ctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = true - defer func() { - s.ctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = false - }() - s.ctx.GetSessionVars().SnapshotInfoschema = s.is - sql := "select * from t where a=1" - comment := fmt.Sprintf("sql:%s", sql) - stmt, err := s.p.ParseOneStmt(sql, "", "") - require.NoError(t, err, comment) - err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) - require.NoError(t, err, comment) - plan := TryFastPlan(s.ctx, stmt) - require.NotNil(t, plan) - require.NotNil(t, s.ctx.GetSessionVars().StmtCtx.OptimizeTracer) - require.NotNil(t, s.ctx.GetSessionVars().StmtCtx.OptimizeTracer.FinalPlan) - require.True(t, s.ctx.GetSessionVars().StmtCtx.OptimizeTracer.IsFastPlan) -} - -func TestWindowLogicalPlanAmbiguous(t *testing.T) { - sql := "select a, max(a) over(), sum(a) over() from t" - var planString string - // The ambiguous logical plan which contains window function can usually be found in 100 iterations. - iterations := 100 - s := createPlannerSuite() - for i := 0; i < iterations; i++ { - stmt, err := s.p.ParseOneStmt(sql, "", "") - require.NoError(t, err) - p, _, err := BuildLogicalPlanForTest(context.Background(), s.ctx, stmt, s.is) - require.NoError(t, err) - if planString == "" { - planString = ToString(p) - } else { - require.Equal(t, ToString(p), planString) - } - } -} - -func TestRemoveOrderbyInSubquery(t *testing.T) { - tests := []struct { - sql string - best string - }{ - { - sql: "select * from t order by a", - best: "DataScan(t)->Projection->Sort", - }, - { - sql: "select (select 1) from t order by a", - best: "DataScan(t)->Projection->Sort->Projection", - }, - { - sql: "select count(*) from (select b from t order by a) n", - best: "DataScan(t)->Projection->Projection->Aggr(count(1),firstrow(test.t.b))->Projection", - }, - { - sql: "select count(1) from (select b from t order by a limit 1) n", - best: "DataScan(t)->Projection->Sort->Limit->Projection->Aggr(count(1),firstrow(test.t.b))->Projection", - }, - } - - s := createPlannerSuite() - s.ctx.GetSessionVars().RemoveOrderbyInSubquery = true - ctx := context.TODO() - for i, tt := range tests { - comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) - stmt, err := s.p.ParseOneStmt(tt.sql, "", "") - require.NoError(t, err, comment) - p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) - require.NoError(t, err, comment) - require.Equal(t, tt.best, ToString(p), comment) - } -} diff --git a/planner/core/logical_plans_test.go b/planner/core/logical_plans_test.go index 2cc0c54d5c10b..82786703338f4 100644 --- a/planner/core/logical_plans_test.go +++ b/planner/core/logical_plans_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 PingCAP, Inc. +// Copyright 2015 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -15,182 +15,2157 @@ package core import ( + "context" "fmt" + "sort" + "strings" "testing" - "github.com/pingcap/errors" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/format" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" - "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/util/hint" "github.com/stretchr/testify/require" ) -func newTypeWithFlen(typeByte byte, flen int) *types.FieldType { - tp := types.NewFieldType(typeByte) - tp.SetFlen(flen) - return tp +type plannerSuite struct { + p *parser.Parser + is infoschema.InfoSchema + ctx sessionctx.Context } -func SubstituteCol2CorCol(expr expression.Expression, colIDs map[int64]struct{}) (expression.Expression, error) { - switch x := expr.(type) { - case *expression.ScalarFunction: - newArgs := make([]expression.Expression, 0, len(x.GetArgs())) - for _, arg := range x.GetArgs() { - newArg, err := SubstituteCol2CorCol(arg, colIDs) - if err != nil { - return nil, errors.Trace(err) - } - newArgs = append(newArgs, newArg) +func createPlannerSuite() (s *plannerSuite) { + s = new(plannerSuite) + tblInfos := []*model.TableInfo{ + MockSignedTable(), + MockUnsignedTable(), + MockView(), + MockNoPKTable(), + MockRangePartitionTable(), + MockHashPartitionTable(), + MockListPartitionTable(), + MockStateNoneColumnTable(), + } + id := int64(0) + for _, tblInfo := range tblInfos { + tblInfo.ID = id + id += 1 + pi := tblInfo.GetPartitionInfo() + if pi == nil { + continue } - newSf, err := expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) - return newSf, errors.Trace(err) - case *expression.Column: - if _, ok := colIDs[x.UniqueID]; ok { - return &expression.CorrelatedColumn{Column: *x}, nil + for _, def := range pi.Definitions { + def.ID = id + id += 1 } } - return expr, nil + s.is = infoschema.MockInfoSchema(tblInfos) + s.ctx = MockContext() + domain.GetDomain(s.ctx).MockInfoCacheAndLoadInfoSchema(s.is) + s.ctx.GetSessionVars().EnableWindowFunction = true + s.p = parser.New() + s.p.SetParserConfig(parser.ParserConfig{EnableWindowFunction: true, EnableStrictDoubleTypeCheck: true}) + return } -func TestIndexPathSplitCorColCond(t *testing.T) { - ctx := MockContext() - totalSchema := expression.NewSchema() - totalSchema.Append(&expression.Column{ - UniqueID: 1, - RetType: types.NewFieldType(mysql.TypeLonglong), - }) - totalSchema.Append(&expression.Column{ - UniqueID: 2, - RetType: types.NewFieldType(mysql.TypeLonglong), - }) - totalSchema.Append(&expression.Column{ - UniqueID: 3, - RetType: newTypeWithFlen(mysql.TypeVarchar, 10), - }) - totalSchema.Append(&expression.Column{ - UniqueID: 4, - RetType: newTypeWithFlen(mysql.TypeVarchar, 10), - }) - totalSchema.Append(&expression.Column{ - UniqueID: 5, - RetType: types.NewFieldType(mysql.TypeLonglong), - }) - names := make(types.NameSlice, 0, 5) - names = append(names, &types.FieldName{ColName: model.NewCIStr("col1")}) - names = append(names, &types.FieldName{ColName: model.NewCIStr("col2")}) - names = append(names, &types.FieldName{ColName: model.NewCIStr("col3")}) - names = append(names, &types.FieldName{ColName: model.NewCIStr("col4")}) - names = append(names, &types.FieldName{ColName: model.NewCIStr("col5")}) - testCases := []struct { - expr string - corColIDs []int64 - idxColIDs []int64 - idxColLens []int - access string - remained string +func TestPredicatePushDown(t *testing.T) { + var input, output []string + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + s := createPlannerSuite() + ctx := context.Background() + for ith, ca := range input { + comment := fmt.Sprintf("for %s", ca) + stmt, err := s.p.ParseOneStmt(ca, "", "") + require.NoError(t, err, comment) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err) + p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) + require.NoError(t, err) + testdata.OnRecord(func() { + output[ith] = ToString(p) + }) + require.Equal(t, output[ith], ToString(p), fmt.Sprintf("for %s %d", ca, ith)) + } +} + +// Issue: 31399 +func TestImplicitCastNotNullFlag(t *testing.T) { + ctx := context.Background() + ca := "select count(*) from t3 group by a having bit_and(b) > 1;" + comment := fmt.Sprintf("for %s", ca) + s := createPlannerSuite() + stmt, err := s.p.ParseOneStmt(ca, "", "") + require.NoError(t, err, comment) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err) + p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagJoinReOrder|flagPrunColumns|flagEliminateProjection, p.(LogicalPlan)) + require.NoError(t, err) + // AggFuncs[0] is count; AggFuncs[1] is bit_and, args[0] is return type of the implicit cast + castNotNullFlag := (p.(*LogicalProjection).children[0].(*LogicalSelection).children[0].(*LogicalAggregation).AggFuncs[1].Args[0].GetType().GetFlag()) & mysql.NotNullFlag + var nullableFlag uint = 0 + require.Equal(t, nullableFlag, castNotNullFlag) +} + +func TestEliminateProjectionUnderUnion(t *testing.T) { + ctx := context.Background() + ca := "Select a from t3 join ( (select 127 as IDD from t3) union all (select 1 as IDD from t3) ) u on t3.b = u.IDD;" + comment := fmt.Sprintf("for %s", ca) + s := createPlannerSuite() + stmt, err := s.p.ParseOneStmt(ca, "", "") + require.NoError(t, err, comment) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err) + p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagJoinReOrder|flagPrunColumns|flagEliminateProjection, p.(LogicalPlan)) + require.NoError(t, err) + // after folding constants, the null flag should keep the same with the old one's (i.e., the schema's). + schemaNullFlag := p.(*LogicalProjection).children[0].(*LogicalJoin).children[1].Children()[1].(*LogicalProjection).schema.Columns[0].RetType.GetFlag() & mysql.NotNullFlag + exprNullFlag := p.(*LogicalProjection).children[0].(*LogicalJoin).children[1].Children()[1].(*LogicalProjection).Exprs[0].GetType().GetFlag() & mysql.NotNullFlag + require.Equal(t, exprNullFlag, schemaNullFlag) +} + +func TestJoinPredicatePushDown(t *testing.T) { + var ( + input []string + output []struct { + Left string + Right string + } + ) + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + + s := createPlannerSuite() + ctx := context.Background() + for i, ca := range input { + comment := fmt.Sprintf("for %s", ca) + stmt, err := s.p.ParseOneStmt(ca, "", "") + require.NoError(t, err, comment) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err, comment) + p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) + require.NoError(t, err, comment) + proj, ok := p.(*LogicalProjection) + require.True(t, ok, comment) + join, ok := proj.children[0].(*LogicalJoin) + require.True(t, ok, comment) + leftPlan, ok := join.children[0].(*DataSource) + require.True(t, ok, comment) + rightPlan, ok := join.children[1].(*DataSource) + require.True(t, ok, comment) + leftCond := fmt.Sprintf("%s", leftPlan.pushedDownConds) + rightCond := fmt.Sprintf("%s", rightPlan.pushedDownConds) + testdata.OnRecord(func() { + output[i].Left, output[i].Right = leftCond, rightCond + }) + require.Equal(t, output[i].Left, leftCond, comment) + require.Equal(t, output[i].Right, rightCond, comment) + } +} + +func TestOuterWherePredicatePushDown(t *testing.T) { + var ( + input []string + output []struct { + Sel string + Left string + Right string + } + ) + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + + s := createPlannerSuite() + ctx := context.Background() + for i, ca := range input { + comment := fmt.Sprintf("for %s", ca) + stmt, err := s.p.ParseOneStmt(ca, "", "") + require.NoError(t, err, comment) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err, comment) + p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) + require.NoError(t, err, comment) + proj, ok := p.(*LogicalProjection) + require.True(t, ok, comment) + selection, ok := proj.children[0].(*LogicalSelection) + require.True(t, ok, comment) + selCond := fmt.Sprintf("%s", selection.Conditions) + testdata.OnRecord(func() { + output[i].Sel = selCond + }) + require.Equal(t, output[i].Sel, selCond, comment) + join, ok := selection.children[0].(*LogicalJoin) + require.True(t, ok, comment) + leftPlan, ok := join.children[0].(*DataSource) + require.True(t, ok, comment) + rightPlan, ok := join.children[1].(*DataSource) + require.True(t, ok, comment) + leftCond := fmt.Sprintf("%s", leftPlan.pushedDownConds) + rightCond := fmt.Sprintf("%s", rightPlan.pushedDownConds) + testdata.OnRecord(func() { + output[i].Left, output[i].Right = leftCond, rightCond + }) + require.Equal(t, output[i].Left, leftCond, comment) + require.Equal(t, output[i].Right, rightCond, comment) + } +} + +func TestSimplifyOuterJoin(t *testing.T) { + var ( + input []string + output []struct { + Best string + JoinType string + } + ) + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + + s := createPlannerSuite() + ctx := context.Background() + for i, ca := range input { + comment := fmt.Sprintf("for %s", ca) + stmt, err := s.p.ParseOneStmt(ca, "", "") + require.NoError(t, err, comment) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err, comment) + p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) + require.NoError(t, err, comment) + planString := ToString(p) + testdata.OnRecord(func() { + output[i].Best = planString + }) + require.Equal(t, output[i].Best, planString, comment) + join, ok := p.(LogicalPlan).Children()[0].(*LogicalJoin) + if !ok { + join, ok = p.(LogicalPlan).Children()[0].Children()[0].(*LogicalJoin) + require.True(t, ok, comment) + } + testdata.OnRecord(func() { + output[i].JoinType = join.JoinType.String() + }) + require.Equal(t, output[i].JoinType, join.JoinType.String(), comment) + } +} + +func TestAntiSemiJoinConstFalse(t *testing.T) { + tests := []struct { + sql string + best string + joinType string + }{ + { + sql: "select a from t t1 where not exists (select a from t t2 where t1.a = t2.a and t2.b = 1 and t2.b = 2)", + best: "Join{DataScan(t1)->DataScan(t2)}(test.t.a,test.t.a)->Projection", + joinType: "anti semi join", + }, + } + + s := createPlannerSuite() + ctx := context.Background() + for _, ca := range tests { + comment := fmt.Sprintf("for %s", ca.sql) + stmt, err := s.p.ParseOneStmt(ca.sql, "", "") + require.NoError(t, err, comment) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err, comment) + p, err = logicalOptimize(context.TODO(), flagDecorrelate|flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) + require.NoError(t, err, comment) + require.Equal(t, ca.best, ToString(p), comment) + join, _ := p.(LogicalPlan).Children()[0].(*LogicalJoin) + require.Equal(t, ca.joinType, join.JoinType.String(), comment) + } +} + +func TestDeriveNotNullConds(t *testing.T) { + var ( + input []string + output []struct { + Plan string + Left string + Right string + } + ) + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + + s := createPlannerSuite() + ctx := context.Background() + for i, ca := range input { + comment := fmt.Sprintf("for %s", ca) + stmt, err := s.p.ParseOneStmt(ca, "", "") + require.NoError(t, err, comment) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err, comment) + p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain|flagDecorrelate, p.(LogicalPlan)) + require.NoError(t, err, comment) + testdata.OnRecord(func() { + output[i].Plan = ToString(p) + }) + require.Equal(t, output[i].Plan, ToString(p), comment) + join := p.(LogicalPlan).Children()[0].(*LogicalJoin) + left := join.Children()[0].(*DataSource) + right := join.Children()[1].(*DataSource) + leftConds := fmt.Sprintf("%s", left.pushedDownConds) + rightConds := fmt.Sprintf("%s", right.pushedDownConds) + testdata.OnRecord(func() { + output[i].Left, output[i].Right = leftConds, rightConds + }) + require.Equal(t, output[i].Left, leftConds, comment) + require.Equal(t, output[i].Right, rightConds, comment) + } +} + +func TestExtraPKNotNullFlag(t *testing.T) { + sql := "select count(*) from t3" + s := createPlannerSuite() + ctx := context.Background() + comment := fmt.Sprintf("for %s", sql) + stmt, err := s.p.ParseOneStmt(sql, "", "") + require.NoError(t, err, comment) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err, comment) + ds := p.(*LogicalProjection).children[0].(*LogicalAggregation).children[0].(*DataSource) + require.Equal(t, "_tidb_rowid", ds.Columns[2].Name.L) + require.Equal(t, mysql.PriKeyFlag|mysql.NotNullFlag, ds.Columns[2].GetFlag()) + require.Equal(t, mysql.PriKeyFlag|mysql.NotNullFlag, ds.schema.Columns[2].RetType.GetFlag()) +} + +func buildLogicPlan4GroupBy(s *plannerSuite, t *testing.T, sql string) (Plan, error) { + sqlMode := s.ctx.GetSessionVars().SQLMode + mockedTableInfo := MockSignedTable() + // mock the table info here for later use + // enable only full group by + s.ctx.GetSessionVars().SQLMode = sqlMode | mysql.ModeOnlyFullGroupBy + defer func() { s.ctx.GetSessionVars().SQLMode = sqlMode }() // restore it + comment := fmt.Sprintf("for %s", sql) + stmt, err := s.p.ParseOneStmt(sql, "", "") + require.NoError(t, err, comment) + + stmt.(*ast.SelectStmt).From.TableRefs.Left.(*ast.TableSource).Source.(*ast.TableName).TableInfo = mockedTableInfo + + p, _, err := BuildLogicalPlanForTest(context.Background(), s.ctx, stmt, s.is) + return p, err +} + +func TestGroupByWhenNotExistCols(t *testing.T) { + sqlTests := []struct { + sql string + expectedErrMatch string }{ { - expr: "col1 = col2", - corColIDs: []int64{2}, - idxColIDs: []int64{1}, - idxColLens: []int{types.UnspecifiedLength}, - access: "[eq(Column#1, Column#2)]", - remained: "[]", + sql: "select a from t group by b", + expectedErrMatch: ".*contains nonaggregated column 'test\\.t\\.a'.*", + }, + { + // has an as column alias + sql: "select a as tempField from t group by b", + expectedErrMatch: ".*contains nonaggregated column 'test\\.t\\.a'.*", + }, + { + // has as table alias + sql: "select tempTable.a from t as tempTable group by b", + expectedErrMatch: ".*contains nonaggregated column 'test\\.tempTable\\.a'.*", + }, + { + // has a func call + sql: "select length(a) from t group by b", + expectedErrMatch: ".*contains nonaggregated column 'test\\.t\\.a'.*", }, { - expr: "col1 = col5 and col2 = 1", - corColIDs: []int64{5}, - idxColIDs: []int64{1, 2}, - idxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength}, - access: "[eq(Column#1, Column#5) eq(Column#2, 1)]", - remained: "[]", + // has a func call with two cols + sql: "select length(b + a) from t group by b", + expectedErrMatch: ".*contains nonaggregated column 'test\\.t\\.a'.*", }, { - expr: "col1 = col5 and col2 = 1", - corColIDs: []int64{5}, - idxColIDs: []int64{2, 1}, - idxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength}, - access: "[]", - remained: "[eq(Column#1, Column#5) eq(Column#2, 1)]", + // has a func call with two cols + sql: "select length(a + b) from t group by b", + expectedErrMatch: ".*contains nonaggregated column 'test\\.t\\.a'.*", }, { - expr: "col1 = col5 and col2 = 1", - corColIDs: []int64{5}, - idxColIDs: []int64{1}, - idxColLens: []int{types.UnspecifiedLength}, - access: "[eq(Column#1, Column#5)]", - remained: "[eq(Column#2, 1)]", + // has a func call with two cols + sql: "select length(a + b) as tempField from t group by b", + expectedErrMatch: ".*contains nonaggregated column 'test\\.t\\.a'.*", + }, + } + s := createPlannerSuite() + for _, test := range sqlTests { + sql := test.sql + p, err := buildLogicPlan4GroupBy(s, t, sql) + require.Nil(t, p) + require.Error(t, err) + require.Regexp(t, test.expectedErrMatch, err.Error()) + } +} + +func TestDupRandJoinCondsPushDown(t *testing.T) { + sql := "select * from t as t1 join t t2 on t1.a > rand() and t1.a > rand()" + comment := fmt.Sprintf("for %s", sql) + s := createPlannerSuite() + stmt, err := s.p.ParseOneStmt(sql, "", "") + require.NoError(t, err, comment) + p, _, err := BuildLogicalPlanForTest(context.Background(), s.ctx, stmt, s.is) + require.NoError(t, err, comment) + p, err = logicalOptimize(context.TODO(), flagPredicatePushDown, p.(LogicalPlan)) + require.NoError(t, err, comment) + proj, ok := p.(*LogicalProjection) + require.True(t, ok, comment) + join, ok := proj.children[0].(*LogicalJoin) + require.True(t, ok, comment) + leftPlan, ok := join.children[0].(*LogicalSelection) + require.True(t, ok, comment) + leftCond := fmt.Sprintf("%s", leftPlan.Conditions) + // Condition with mutable function cannot be de-duplicated when push down join conds. + require.Equal(t, "[gt(cast(test.t.a, double BINARY), rand()) gt(cast(test.t.a, double BINARY), rand())]", leftCond, comment) +} + +func TestTablePartition(t *testing.T) { + variable.EnableMDL.Store(false) + definitions := []model.PartitionDefinition{ + { + ID: 41, + Name: model.NewCIStr("p1"), + LessThan: []string{"16"}, }, { - expr: "col2 = 1 and col1 = col5", - corColIDs: []int64{5}, - idxColIDs: []int64{1}, - idxColLens: []int{types.UnspecifiedLength}, - access: "[eq(Column#1, Column#5)]", - remained: "[eq(Column#2, 1)]", + ID: 42, + Name: model.NewCIStr("p2"), + LessThan: []string{"32"}, }, { - expr: "col1 = col2 and col3 = col4 and col5 = 1", - corColIDs: []int64{2, 4}, - idxColIDs: []int64{1, 3}, - idxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength}, - access: "[eq(Column#1, Column#2) eq(Column#3, Column#4)]", - remained: "[eq(Column#5, 1)]", + ID: 43, + Name: model.NewCIStr("p3"), + LessThan: []string{"64"}, }, { - expr: "col1 = col2 and col3 = col4 and col5 = 1", - corColIDs: []int64{2, 4}, - idxColIDs: []int64{1, 3}, - idxColLens: []int{types.UnspecifiedLength, 2}, - access: "[eq(Column#1, Column#2) eq(Column#3, Column#4)]", - remained: "[eq(Column#3, Column#4) eq(Column#5, 1)]", + ID: 44, + Name: model.NewCIStr("p4"), + LessThan: []string{"128"}, }, { - expr: `col1 = col5 and col3 = "col1" and col2 = col5`, - corColIDs: []int64{5}, - idxColIDs: []int64{1, 2, 3}, - idxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength, types.UnspecifiedLength}, - access: "[eq(Column#1, Column#5) eq(Column#2, Column#5) eq(Column#3, col1)]", - remained: "[]", + ID: 45, + Name: model.NewCIStr("p5"), + LessThan: []string{"maxvalue"}, }, + } + is := MockPartitionInfoSchema(definitions) + // is1 equals to is without maxvalue partition. + definitions1 := make([]model.PartitionDefinition, len(definitions)-1) + copy(definitions1, definitions) + is1 := MockPartitionInfoSchema(definitions1) + isChoices := []infoschema.InfoSchema{is, is1} + + var ( + input []struct { + SQL string + IsIdx int + } + output []string + ) + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + + s := createPlannerSuite() + ctx := context.Background() + for i, ca := range input { + comment := fmt.Sprintf("for %s", ca.SQL) + stmt, err := s.p.ParseOneStmt(ca.SQL, "", "") + require.NoError(t, err, comment) + testdata.OnRecord(func() { + + }) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, isChoices[ca.IsIdx]) + require.NoError(t, err) + p, err = logicalOptimize(context.TODO(), flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain|flagPredicatePushDown|flagPartitionProcessor, p.(LogicalPlan)) + require.NoError(t, err) + planString := ToString(p) + testdata.OnRecord(func() { + output[i] = planString + }) + require.Equal(t, output[i], ToString(p), fmt.Sprintf("for %v", ca)) + } +} + +func TestSubquery(t *testing.T) { + var input, output []string + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + + s := createPlannerSuite() + ctx := context.Background() + for ith, ca := range input { + comment := fmt.Sprintf("for %s", ca) + stmt, err := s.p.ParseOneStmt(ca, "", "") + require.NoError(t, err, comment) + + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err) + if lp, ok := p.(LogicalPlan); ok { + p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain|flagSemiJoinRewrite, lp) + require.NoError(t, err) + } + testdata.OnRecord(func() { + output[ith] = ToString(p) + }) + require.Equal(t, output[ith], ToString(p), fmt.Sprintf("for %s %d", ca, ith)) + } +} + +func TestPlanBuilder(t *testing.T) { + var input, output []string + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + + s := createPlannerSuite() + s.ctx.GetSessionVars().CostModelVersion = modelVer1 + ctx := context.Background() + for i, ca := range input { + comment := fmt.Sprintf("for %s", ca) + stmt, err := s.p.ParseOneStmt(ca, "", "") + require.NoError(t, err, comment) + + s.ctx.GetSessionVars().SetHashJoinConcurrency(1) + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err) + if lp, ok := p.(LogicalPlan); ok { + p, err = logicalOptimize(context.TODO(), flagPrunColumns|flagPrunColumnsAgain, lp) + require.NoError(t, err) + } + testdata.OnRecord(func() { + output[i] = ToString(p) + }) + require.Equal(t, output[i], ToString(p), fmt.Sprintf("for %s", ca)) + } +} + +func TestJoinReOrder(t *testing.T) { + var input, output []string + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + + s := createPlannerSuite() + ctx := context.Background() + for i, tt := range input { + comment := fmt.Sprintf("for %s", tt) + stmt, err := s.p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err) + p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagJoinReOrder, p.(LogicalPlan)) + require.NoError(t, err) + planString := ToString(p) + testdata.OnRecord(func() { + output[i] = planString + }) + require.Equal(t, output[i], planString, fmt.Sprintf("for %s", tt)) + } +} + +func TestEagerAggregation(t *testing.T) { + var input []string + var output []string + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + + s := createPlannerSuite() + ctx := context.Background() + s.ctx.GetSessionVars().AllowAggPushDown = true + defer func() { + s.ctx.GetSessionVars().AllowAggPushDown = false + }() + for ith, tt := range input { + comment := fmt.Sprintf("for %s", tt) + stmt, err := s.p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err) + p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain|flagPushDownAgg, p.(LogicalPlan)) + require.NoError(t, err) + testdata.OnRecord(func() { + output[ith] = ToString(p) + }) + require.Equal(t, output[ith], ToString(p), fmt.Sprintf("for %s %d", tt, ith)) + } +} + +func TestColumnPruning(t *testing.T) { + var ( + input []string + output []map[int][]string + ) + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + + s := createPlannerSuite() + ctx := context.Background() + for i, tt := range input { + comment := fmt.Sprintf("case:%v sql:\"%s\"", i, tt) + stmt, err := s.p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err) + lp, err := logicalOptimize(ctx, flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i] = make(map[int][]string) + }) + checkDataSourceCols(lp, t, output[i], comment) + } +} + +func TestSortByItemsPruning(t *testing.T) { + var ( + input []string + output [][]string + ) + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + testdata.OnRecord(func() { + output = make([][]string, len(input)) + }) + + s := createPlannerSuite() + ctx := context.Background() + for i, tt := range input { + comment := fmt.Sprintf("for %s", tt) + stmt, err := s.p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err) + lp, err := logicalOptimize(ctx, flagEliminateProjection|flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) + require.NoError(t, err) + checkOrderByItems(lp, t, &output[i], comment) + } +} + +func TestProjectionEliminator(t *testing.T) { + tests := []struct { + sql string + best string + }{ { - expr: "col1 = col5 and col3 = CHAR(1 COLLATE 'binary')", - corColIDs: []int64{5}, - idxColIDs: []int64{1, 3}, - idxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength}, - access: "[eq(Column#1, Column#5) eq(Column#3, \x01)]", - remained: "[]", + sql: "select 1+num from (select 1+a as num from t) t1;", + best: "DataScan(t)->Projection", + }, { + sql: "select count(*) from t where a in (select b from t2 where a is null);", + best: "Join{DataScan(t)->Dual->Aggr(firstrow(test.t2.b))}(test.t.a,test.t2.b)->Aggr(count(1))->Projection", }, } - for _, tt := range testCases { - comment := fmt.Sprintf("failed at case:\nexpr: %v\ncorColIDs: %v\nidxColIDs: %v\nidxColLens: %v\naccess: %v\nremained: %v\n", tt.expr, tt.corColIDs, tt.idxColIDs, tt.idxColLens, tt.access, tt.remained) - filters, err := expression.ParseSimpleExprsWithNames(ctx, tt.expr, totalSchema, names) + + s := createPlannerSuite() + ctx := context.Background() + for ith, tt := range tests { + comment := fmt.Sprintf("for %s", tt.sql) + stmt, err := s.p.ParseOneStmt(tt.sql, "", "") require.NoError(t, err, comment) - if sf, ok := filters[0].(*expression.ScalarFunction); ok && sf.FuncName.L == ast.LogicAnd { - filters = expression.FlattenCNFConditions(sf) + + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err) + p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagPrunColumns|flagPrunColumnsAgain|flagEliminateProjection, p.(LogicalPlan)) + require.NoError(t, err) + require.Equal(t, tt.best, ToString(p), fmt.Sprintf("for %s %d", tt.sql, ith)) + } +} + +func TestCS3389(t *testing.T) { + s := createPlannerSuite() + ctx := context.Background() + stmt, err := s.p.ParseOneStmt("select count(*) from t where a in (select b from t2 where a is null);", "", "") + require.NoError(t, err) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err) + p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagPrunColumns|flagPrunColumnsAgain|flagEliminateProjection|flagJoinReOrder, p.(LogicalPlan)) + require.NoError(t, err) + + // Assert that all Projection is not empty and there is no Projection between Aggregation and Join. + proj, isProj := p.(*LogicalProjection) + require.True(t, isProj) + require.True(t, len(proj.Exprs) > 0) + child := proj.Children()[0] + agg, isAgg := child.(*LogicalAggregation) + require.True(t, isAgg) + child = agg.Children()[0] + _, isJoin := child.(*LogicalJoin) + require.True(t, isJoin) +} + +func TestAllocID(t *testing.T) { + ctx := MockContext() + pA := DataSource{}.Init(ctx, 0) + pB := DataSource{}.Init(ctx, 0) + require.Equal(t, pB.id, pA.id+1) +} + +func checkDataSourceCols(p LogicalPlan, t *testing.T, ans map[int][]string, comment string) { + switch v := p.(type) { + case *DataSource, *LogicalUnionAll, *LogicalLimit: + testdata.OnRecord(func() { + ans[p.ID()] = make([]string, p.Schema().Len()) + }) + colList, ok := ans[p.ID()] + require.True(t, ok, fmt.Sprintf("For %s %T ID %d Not found", comment, v, p.ID())) + require.Equal(t, len(colList), len(p.Schema().Columns), comment) + for i, col := range p.Schema().Columns { + testdata.OnRecord(func() { + colList[i] = col.String() + }) + require.Equal(t, colList[i], col.String(), comment) } - trueFilters := make([]expression.Expression, 0, len(filters)) - idMap := make(map[int64]struct{}) - for _, id := range tt.corColIDs { - idMap[id] = struct{}{} + } + for _, child := range p.Children() { + checkDataSourceCols(child, t, ans, comment) + } +} + +func checkOrderByItems(p LogicalPlan, t *testing.T, colList *[]string, comment string) { + switch p := p.(type) { + case *LogicalSort: + testdata.OnRecord(func() { + *colList = make([]string, len(p.ByItems)) + }) + for i, col := range p.ByItems { + testdata.OnRecord(func() { + (*colList)[i] = col.String() + }) + s := col.String() + require.Equal(t, (*colList)[i], s, comment) } - for _, filter := range filters { - trueFilter, err := SubstituteCol2CorCol(filter, idMap) + } + children := p.Children() + require.LessOrEqual(t, len(children), 1, fmt.Sprintf("For %v Expected <= 1 Child", comment)) + for _, child := range children { + checkOrderByItems(child, t, colList, comment) + } +} + +func TestValidate(t *testing.T) { + tests := []struct { + sql string + err *terror.Error + }{ + { + sql: "select date_format((1,2), '%H');", + err: expression.ErrOperandColumns, + }, + { + sql: "select cast((1,2) as date)", + err: expression.ErrOperandColumns, + }, + { + sql: "select (1,2) between (3,4) and (5,6)", + err: expression.ErrOperandColumns, + }, + { + sql: "select (1,2) rlike '1'", + err: expression.ErrOperandColumns, + }, + { + sql: "select (1,2) like '1'", + err: expression.ErrOperandColumns, + }, + { + sql: "select case(1,2) when(1,2) then true end", + err: expression.ErrOperandColumns, + }, + { + sql: "select (1,2) in ((3,4),(5,6))", + err: nil, + }, + { + sql: "select row(1,(2,3)) in (select a,b from t)", + err: expression.ErrOperandColumns, + }, + { + sql: "select row(1,2) in (select a,b from t)", + err: nil, + }, + { + sql: "select (1,2) in ((3,4),5)", + err: expression.ErrOperandColumns, + }, + { + sql: "select (1,2) is true", + err: expression.ErrOperandColumns, + }, + { + sql: "select (1,2) is null", + err: expression.ErrOperandColumns, + }, + { + sql: "select (+(1,2))=(1,2)", + err: nil, + }, + { + sql: "select (-(1,2))=(1,2)", + err: expression.ErrOperandColumns, + }, + { + sql: "select (1,2)||(1,2)", + err: expression.ErrOperandColumns, + }, + { + sql: "select (1,2) < (3,4)", + err: nil, + }, + { + sql: "select (1,2) < 3", + err: expression.ErrOperandColumns, + }, + { + sql: "select 1, * from t", + err: ErrInvalidWildCard, + }, + { + sql: "select *, 1 from t", + err: nil, + }, + { + sql: "select 1, t.* from t", + err: nil, + }, + { + sql: "select 1 from t t1, t t2 where t1.a > all((select a) union (select a))", + err: ErrAmbiguous, + }, + { + sql: "insert into t set a = 1, b = a + 1", + err: nil, + }, + { + sql: "insert into t set a = 1, b = values(a) + 1", + err: nil, + }, + { + sql: "select a, b, c from t order by 0", + err: ErrUnknownColumn, + }, + { + sql: "select a, b, c from t order by 4", + err: ErrUnknownColumn, + }, + { + sql: "select a as c1, b as c1 from t order by c1", + err: ErrAmbiguous, + }, + { + sql: "(select a as b, b from t) union (select a, b from t) order by b", + err: ErrAmbiguous, + }, + { + sql: "(select a as b, b from t) union (select a, b from t) order by a", + err: ErrUnknownColumn, + }, + { + sql: "select * from t t1 use index(x)", + err: ErrKeyDoesNotExist, + }, + { + sql: "select a from t having c2", + err: ErrUnknownColumn, + }, + { + sql: "select a from t group by c2 + 1 having c2", + err: ErrUnknownColumn, + }, + { + sql: "select a as b, b from t having b", + err: ErrAmbiguous, + }, + { + sql: "select a + 1 from t having a", + err: ErrUnknownColumn, + }, + { // issue (#20509) + sql: "select * from t left join t2 on t.a=t2.a having not (t.a <=> t2.a)", + err: nil, + }, + { + sql: "select a from t having sum(avg(a))", + err: ErrInvalidGroupFuncUse, + }, + { + sql: "select concat(c_str, d_str) from t group by `concat(c_str, d_str)`", + err: nil, + }, + { + sql: "select concat(c_str, d_str) from t group by `concat(c_str,d_str)`", + err: ErrUnknownColumn, + }, + { + sql: "select a from t b having b.a", + err: nil, + }, + { + sql: "select b.a from t b having b.a", + err: nil, + }, + { + sql: "select b.a from t b having a", + err: nil, + }, + { + sql: "select a+1 from t having t.a", + err: ErrUnknownColumn, + }, + { + sql: "update T_StateNoneColumn set c = 1 where a = 1", + err: ErrUnknownColumn, + }, + } + + s := createPlannerSuite() + ctx := context.Background() + for _, tt := range tests { + sql := tt.sql + comment := fmt.Sprintf("for %s", sql) + stmt, err := s.p.ParseOneStmt(sql, "", "") + require.NoError(t, err, comment) + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err) + _, _, err = BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + if tt.err == nil { require.NoError(t, err, comment) - trueFilters = append(trueFilters, trueFilter) + } else { + require.True(t, tt.err.Equal(err), comment) } - path := util.AccessPath{ - EqCondCount: 0, - TableFilters: trueFilters, - IdxCols: expression.FindPrefixOfIndex(totalSchema.Columns, tt.idxColIDs), - IdxColLens: tt.idxColLens, + } +} + +func checkUniqueKeys(p LogicalPlan, t *testing.T, ans map[int][][]string, sql string) { + testdata.OnRecord(func() { + ans[p.ID()] = make([][]string, len(p.Schema().Keys)) + }) + keyList, ok := ans[p.ID()] + require.True(t, ok, fmt.Sprintf("for %s, %v not found", sql, p.ID())) + require.Equal(t, len(keyList), len(p.Schema().Keys), fmt.Sprintf("for %s, %v, the number of key doesn't match, the schema is %s", sql, p.ID(), p.Schema())) + for i := range keyList { + testdata.OnRecord(func() { + keyList[i] = make([]string, len(p.Schema().Keys[i])) + }) + require.Equal(t, len(keyList[i]), len(p.Schema().Keys[i]), fmt.Sprintf("for %s, %v %v, the number of column doesn't match", sql, p.ID(), keyList[i])) + for j := range keyList[i] { + testdata.OnRecord(func() { + keyList[i][j] = p.Schema().Keys[i][j].String() + }) + require.Equal(t, keyList[i][j], p.Schema().Keys[i][j].String(), fmt.Sprintf("for %s, %v %v, column dosen't match", sql, p.ID(), keyList[i])) } + } + testdata.OnRecord(func() { + ans[p.ID()] = keyList + }) + for _, child := range p.Children() { + checkUniqueKeys(child, t, ans, sql) + } +} + +func TestUniqueKeyInfo(t *testing.T) { + var input []string + var output []map[int][][]string + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + testdata.OnRecord(func() { + output = make([]map[int][][]string, len(input)) + }) + + s := createPlannerSuite() + ctx := context.Background() + for ith, tt := range input { + comment := fmt.Sprintf("for %s %d", tt, ith) + stmt, err := s.p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err) + lp, err := logicalOptimize(context.TODO(), flagPredicatePushDown|flagPrunColumns|flagBuildKeyInfo, p.(LogicalPlan)) + require.NoError(t, err) + testdata.OnRecord(func() { + output[ith] = make(map[int][][]string) + }) + checkUniqueKeys(lp, t, output[ith], tt) + } +} + +func TestAggPrune(t *testing.T) { + var input, output []string + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + + s := createPlannerSuite() + ctx := context.Background() + for i, tt := range input { + comment := fmt.Sprintf("for %s", tt) + stmt, err := s.p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + domain.GetDomain(s.ctx).MockInfoCacheAndLoadInfoSchema(s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err) + + p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain|flagBuildKeyInfo|flagEliminateAgg|flagEliminateProjection, p.(LogicalPlan)) + require.NoError(t, err) + planString := ToString(p) + testdata.OnRecord(func() { + output[i] = planString + }) + require.Equal(t, output[i], planString, comment) + } +} - access, remained := path.SplitCorColAccessCondFromFilters(ctx, path.EqCondCount) - require.Equal(t, tt.access, fmt.Sprintf("%s", access), comment) - require.Equal(t, tt.remained, fmt.Sprintf("%s", remained), comment) +func TestVisitInfo(t *testing.T) { + variable.EnableMDL.Store(false) + tests := []struct { + sql string + ans []visitInfo + }{ + { + sql: "insert into t (a) values (1)", + ans: []visitInfo{ + {mysql.InsertPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "delete from t where a = 1", + ans: []visitInfo{ + {mysql.DeletePriv, "test", "t", "", nil, false, "", false}, + {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "delete from t order by a", + ans: []visitInfo{ + {mysql.DeletePriv, "test", "t", "", nil, false, "", false}, + {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "delete from t", + ans: []visitInfo{ + {mysql.DeletePriv, "test", "t", "", nil, false, "", false}, + }, + }, + /* Not currently supported. See https://github.com/pingcap/tidb/issues/23644 + { + sql: "delete from t where 1=1", + ans: []visitInfo{ + {mysql.DeletePriv, "test", "t", "", nil, false, "", false}, + }, + }, + */ + { + sql: "delete from a1 using t as a1 inner join t as a2 where a1.a = a2.a", + ans: []visitInfo{ + {mysql.DeletePriv, "test", "t", "", nil, false, "", false}, + {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "update t set a = 7 where a = 1", + ans: []visitInfo{ + {mysql.UpdatePriv, "test", "t", "", nil, false, "", false}, + {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "update t, (select * from t) a1 set t.a = a1.a;", + ans: []visitInfo{ + {mysql.UpdatePriv, "test", "t", "", nil, false, "", false}, + {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "update t a1 set a1.a = a1.a + 1", + ans: []visitInfo{ + {mysql.UpdatePriv, "test", "t", "", nil, false, "", false}, + {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "select a, sum(e) from t group by a", + ans: []visitInfo{ + {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "truncate table t", + ans: []visitInfo{ + {mysql.DropPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "drop table t", + ans: []visitInfo{ + {mysql.DropPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "create table t (a int)", + ans: []visitInfo{ + {mysql.CreatePriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "create table t1 like t", + ans: []visitInfo{ + {mysql.CreatePriv, "test", "t1", "", nil, false, "", false}, + {mysql.SelectPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "create database test", + ans: []visitInfo{ + {mysql.CreatePriv, "test", "", "", nil, false, "", false}, + }, + }, + { + sql: "drop database test", + ans: []visitInfo{ + {mysql.DropPriv, "test", "", "", nil, false, "", false}, + }, + }, + { + sql: "create index t_1 on t (a)", + ans: []visitInfo{ + {mysql.IndexPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "drop index e on t", + ans: []visitInfo{ + {mysql.IndexPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: `grant all privileges on test.* to 'test'@'%'`, + ans: []visitInfo{ + {mysql.SelectPriv, "test", "", "", nil, false, "", false}, + {mysql.InsertPriv, "test", "", "", nil, false, "", false}, + {mysql.UpdatePriv, "test", "", "", nil, false, "", false}, + {mysql.DeletePriv, "test", "", "", nil, false, "", false}, + {mysql.CreatePriv, "test", "", "", nil, false, "", false}, + {mysql.DropPriv, "test", "", "", nil, false, "", false}, + {mysql.GrantPriv, "test", "", "", nil, false, "", false}, + {mysql.ReferencesPriv, "test", "", "", nil, false, "", false}, + {mysql.LockTablesPriv, "test", "", "", nil, false, "", false}, + {mysql.CreateTMPTablePriv, "test", "", "", nil, false, "", false}, + {mysql.EventPriv, "test", "", "", nil, false, "", false}, + {mysql.CreateRoutinePriv, "test", "", "", nil, false, "", false}, + {mysql.AlterRoutinePriv, "test", "", "", nil, false, "", false}, + {mysql.AlterPriv, "test", "", "", nil, false, "", false}, + {mysql.ExecutePriv, "test", "", "", nil, false, "", false}, + {mysql.IndexPriv, "test", "", "", nil, false, "", false}, + {mysql.CreateViewPriv, "test", "", "", nil, false, "", false}, + {mysql.ShowViewPriv, "test", "", "", nil, false, "", false}, + {mysql.TriggerPriv, "test", "", "", nil, false, "", false}, + }, + }, + { + sql: `grant all privileges on *.* to 'test'@'%'`, + ans: []visitInfo{ + {mysql.SelectPriv, "", "", "", nil, false, "", false}, + {mysql.InsertPriv, "", "", "", nil, false, "", false}, + {mysql.UpdatePriv, "", "", "", nil, false, "", false}, + {mysql.DeletePriv, "", "", "", nil, false, "", false}, + {mysql.CreatePriv, "", "", "", nil, false, "", false}, + {mysql.DropPriv, "", "", "", nil, false, "", false}, + {mysql.ProcessPriv, "", "", "", nil, false, "", false}, + {mysql.ReferencesPriv, "", "", "", nil, false, "", false}, + {mysql.AlterPriv, "", "", "", nil, false, "", false}, + {mysql.ShowDBPriv, "", "", "", nil, false, "", false}, + {mysql.SuperPriv, "", "", "", nil, false, "", false}, + {mysql.ExecutePriv, "", "", "", nil, false, "", false}, + {mysql.IndexPriv, "", "", "", nil, false, "", false}, + {mysql.CreateUserPriv, "", "", "", nil, false, "", false}, + {mysql.CreateTablespacePriv, "", "", "", nil, false, "", false}, + {mysql.TriggerPriv, "", "", "", nil, false, "", false}, + {mysql.CreateViewPriv, "", "", "", nil, false, "", false}, + {mysql.ShowViewPriv, "", "", "", nil, false, "", false}, + {mysql.CreateRolePriv, "", "", "", nil, false, "", false}, + {mysql.DropRolePriv, "", "", "", nil, false, "", false}, + {mysql.CreateTMPTablePriv, "", "", "", nil, false, "", false}, + {mysql.LockTablesPriv, "", "", "", nil, false, "", false}, + {mysql.CreateRoutinePriv, "", "", "", nil, false, "", false}, + {mysql.AlterRoutinePriv, "", "", "", nil, false, "", false}, + {mysql.EventPriv, "", "", "", nil, false, "", false}, + {mysql.ShutdownPriv, "", "", "", nil, false, "", false}, + {mysql.ReloadPriv, "", "", "", nil, false, "", false}, + {mysql.FilePriv, "", "", "", nil, false, "", false}, + {mysql.ConfigPriv, "", "", "", nil, false, "", false}, + {mysql.ReplicationClientPriv, "", "", "", nil, false, "", false}, + {mysql.ReplicationSlavePriv, "", "", "", nil, false, "", false}, + {mysql.GrantPriv, "", "", "", nil, false, "", false}, + }, + }, + { + sql: `grant select on test.ttt to 'test'@'%'`, + ans: []visitInfo{ + {mysql.SelectPriv, "test", "ttt", "", nil, false, "", false}, + {mysql.GrantPriv, "test", "ttt", "", nil, false, "", false}, + }, + }, + { + sql: `grant select on ttt to 'test'@'%'`, + ans: []visitInfo{ + {mysql.SelectPriv, "test", "ttt", "", nil, false, "", false}, + {mysql.GrantPriv, "test", "ttt", "", nil, false, "", false}, + }, + }, + { + sql: `revoke all privileges on test.* from 'test'@'%'`, + ans: []visitInfo{ + {mysql.SelectPriv, "test", "", "", nil, false, "", false}, + {mysql.InsertPriv, "test", "", "", nil, false, "", false}, + {mysql.UpdatePriv, "test", "", "", nil, false, "", false}, + {mysql.DeletePriv, "test", "", "", nil, false, "", false}, + {mysql.CreatePriv, "test", "", "", nil, false, "", false}, + {mysql.DropPriv, "test", "", "", nil, false, "", false}, + {mysql.GrantPriv, "test", "", "", nil, false, "", false}, + {mysql.ReferencesPriv, "test", "", "", nil, false, "", false}, + {mysql.LockTablesPriv, "test", "", "", nil, false, "", false}, + {mysql.CreateTMPTablePriv, "test", "", "", nil, false, "", false}, + {mysql.EventPriv, "test", "", "", nil, false, "", false}, + {mysql.CreateRoutinePriv, "test", "", "", nil, false, "", false}, + {mysql.AlterRoutinePriv, "test", "", "", nil, false, "", false}, + {mysql.AlterPriv, "test", "", "", nil, false, "", false}, + {mysql.ExecutePriv, "test", "", "", nil, false, "", false}, + {mysql.IndexPriv, "test", "", "", nil, false, "", false}, + {mysql.CreateViewPriv, "test", "", "", nil, false, "", false}, + {mysql.ShowViewPriv, "test", "", "", nil, false, "", false}, + {mysql.TriggerPriv, "test", "", "", nil, false, "", false}, + }, + }, + { + sql: `revoke connection_admin on *.* from u1`, + ans: []visitInfo{ + {mysql.ExtendedPriv, "", "", "", nil, false, "CONNECTION_ADMIN", true}, + }, + }, + { + sql: `revoke connection_admin, select on *.* from u1`, + ans: []visitInfo{ + {mysql.ExtendedPriv, "", "", "", nil, false, "CONNECTION_ADMIN", true}, + {mysql.SelectPriv, "", "", "", nil, false, "", false}, + {mysql.GrantPriv, "", "", "", nil, false, "", false}, + }, + }, + { + sql: `revoke all privileges on *.* FROM u1`, + ans: []visitInfo{ + {mysql.SelectPriv, "", "", "", nil, false, "", false}, + {mysql.InsertPriv, "", "", "", nil, false, "", false}, + {mysql.UpdatePriv, "", "", "", nil, false, "", false}, + {mysql.DeletePriv, "", "", "", nil, false, "", false}, + {mysql.CreatePriv, "", "", "", nil, false, "", false}, + {mysql.DropPriv, "", "", "", nil, false, "", false}, + {mysql.ProcessPriv, "", "", "", nil, false, "", false}, + {mysql.ReferencesPriv, "", "", "", nil, false, "", false}, + {mysql.AlterPriv, "", "", "", nil, false, "", false}, + {mysql.ShowDBPriv, "", "", "", nil, false, "", false}, + {mysql.SuperPriv, "", "", "", nil, false, "", false}, + {mysql.ExecutePriv, "", "", "", nil, false, "", false}, + {mysql.IndexPriv, "", "", "", nil, false, "", false}, + {mysql.CreateUserPriv, "", "", "", nil, false, "", false}, + {mysql.CreateTablespacePriv, "", "", "", nil, false, "", false}, + {mysql.TriggerPriv, "", "", "", nil, false, "", false}, + {mysql.CreateViewPriv, "", "", "", nil, false, "", false}, + {mysql.ShowViewPriv, "", "", "", nil, false, "", false}, + {mysql.CreateRolePriv, "", "", "", nil, false, "", false}, + {mysql.DropRolePriv, "", "", "", nil, false, "", false}, + {mysql.CreateTMPTablePriv, "", "", "", nil, false, "", false}, + {mysql.LockTablesPriv, "", "", "", nil, false, "", false}, + {mysql.CreateRoutinePriv, "", "", "", nil, false, "", false}, + {mysql.AlterRoutinePriv, "", "", "", nil, false, "", false}, + {mysql.EventPriv, "", "", "", nil, false, "", false}, + {mysql.ShutdownPriv, "", "", "", nil, false, "", false}, + {mysql.ReloadPriv, "", "", "", nil, false, "", false}, + {mysql.FilePriv, "", "", "", nil, false, "", false}, + {mysql.ConfigPriv, "", "", "", nil, false, "", false}, + {mysql.ReplicationClientPriv, "", "", "", nil, false, "", false}, + {mysql.ReplicationSlavePriv, "", "", "", nil, false, "", false}, + {mysql.GrantPriv, "", "", "", nil, false, "", false}, + }, + }, + { + sql: `set password for 'root'@'%' = 'xxxxx'`, + ans: []visitInfo{}, + }, + { + sql: `show create table test.ttt`, + ans: []visitInfo{ + {mysql.AllPrivMask, "test", "ttt", "", nil, false, "", false}, + }, + }, + { + sql: "alter table t add column a int(4)", + ans: []visitInfo{ + {mysql.AlterPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "rename table t_old to t_new", + ans: []visitInfo{ + {mysql.AlterPriv, "test", "t_old", "", nil, false, "", false}, + {mysql.DropPriv, "test", "t_old", "", nil, false, "", false}, + {mysql.CreatePriv, "test", "t_new", "", nil, false, "", false}, + {mysql.InsertPriv, "test", "t_new", "", nil, false, "", false}, + }, + }, + { + sql: "alter table t_old rename to t_new", + ans: []visitInfo{ + {mysql.AlterPriv, "test", "t_old", "", nil, false, "", false}, + {mysql.DropPriv, "test", "t_old", "", nil, false, "", false}, + {mysql.CreatePriv, "test", "t_new", "", nil, false, "", false}, + {mysql.InsertPriv, "test", "t_new", "", nil, false, "", false}, + }, + }, + { + sql: "alter table t drop partition p0;", + ans: []visitInfo{ + {mysql.AlterPriv, "test", "t", "", nil, false, "", false}, + {mysql.DropPriv, "test", "t", "", nil, false, "", false}, + }, + }, + { + sql: "flush privileges", + ans: []visitInfo{ + {mysql.ReloadPriv, "", "", "", ErrSpecificAccessDenied, false, "", false}, + }, + }, + { + sql: "SET GLOBAL wait_timeout=12345", + ans: []visitInfo{ + {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "SYSTEM_VARIABLES_ADMIN", false}, + }, + }, + { + sql: "create placement policy x LEARNERS=1", + ans: []visitInfo{ + {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "PLACEMENT_ADMIN", false}, + }, + }, + { + sql: "drop placement policy if exists x", + ans: []visitInfo{ + {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "PLACEMENT_ADMIN", false}, + }, + }, + { + sql: "BACKUP DATABASE test TO 'local:///tmp/a'", + ans: []visitInfo{ + {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "BACKUP_ADMIN", false}, + }, + }, + { + sql: "RESTORE DATABASE test FROM 'local:///tmp/a'", + ans: []visitInfo{ + {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "RESTORE_ADMIN", false}, + }, + }, + { + sql: "SHOW BACKUPS", + ans: []visitInfo{ + {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "BACKUP_ADMIN", false}, + }, + }, + { + sql: "SHOW RESTORES", + ans: []visitInfo{ + {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "RESTORE_ADMIN", false}, + }, + }, + { + sql: "GRANT rolename TO user1", + ans: []visitInfo{ + {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "ROLE_ADMIN", false}, + }, + }, + { + sql: "REVOKE rolename FROM user1", + ans: []visitInfo{ + {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "ROLE_ADMIN", false}, + }, + }, + { + sql: "GRANT BACKUP_ADMIN ON *.* TO user1", + ans: []visitInfo{ + {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "BACKUP_ADMIN", true}, + }, + }, + { + sql: "GRANT BACKUP_ADMIN ON *.* TO user1 WITH GRANT OPTION", + ans: []visitInfo{ + {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "BACKUP_ADMIN", true}, + }, + }, + { + sql: "RENAME USER user1 to user1_tmp", + ans: []visitInfo{ + {mysql.CreateUserPriv, "", "", "", ErrSpecificAccessDenied, false, "", false}, + }, + }, + { + sql: "SHOW CONFIG", + ans: []visitInfo{ + {mysql.ConfigPriv, "", "", "", ErrSpecificAccessDenied, false, "", false}, + }, + }, + } + + s := createPlannerSuite() + for _, tt := range tests { + comment := fmt.Sprintf("for %s", tt.sql) + stmt, err := s.p.ParseOneStmt(tt.sql, "", "") + require.NoError(t, err, comment) + + // TODO: to fix, Table 'test.ttt' doesn't exist + _ = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + sctx := MockContext() + builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) + domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) + builder.ctx.GetSessionVars().SetHashJoinConcurrency(1) + _, err = builder.Build(context.TODO(), stmt) + require.NoError(t, err, comment) + + checkVisitInfo(t, builder.visitInfo, tt.ans, comment) + } +} + +type visitInfoArray []visitInfo + +func (v visitInfoArray) Len() int { + return len(v) +} + +func (v visitInfoArray) Less(i, j int) bool { + if v[i].privilege < v[j].privilege { + return true + } + if v[i].db < v[j].db { + return true + } + if v[i].table < v[j].table { + return true + } + if v[i].column < v[j].column { + return true + } + + return false +} + +func (v visitInfoArray) Swap(i, j int) { + v[i], v[j] = v[j], v[i] +} + +func unique(v []visitInfo) []visitInfo { + repeat := 0 + for i := 1; i < len(v); i++ { + if v[i] == v[i-1] { + repeat++ + } else { + v[i-repeat] = v[i] + } + } + return v[:len(v)-repeat] +} + +func checkVisitInfo(t *testing.T, v1, v2 []visitInfo, comment string) { + sort.Sort(visitInfoArray(v1)) + sort.Sort(visitInfoArray(v2)) + v1 = unique(v1) + v2 = unique(v2) + + require.Equal(t, len(v2), len(v1), comment) + for i := 0; i < len(v1); i++ { + // loose compare errors for code match + require.True(t, terror.ErrorEqual(v1[i].err, v2[i].err), fmt.Sprintf("err1 %v, err2 %v for %s", v1[i].err, v2[i].err, comment)) + // compare remainder + v1[i].err = v2[i].err + require.Equal(t, v2[i], v1[i], comment) + } +} + +func TestUnion(t *testing.T) { + var input []string + var output []struct { + Best string + Err bool + } + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + s := createPlannerSuite() + ctx := context.TODO() + for i, tt := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, tt) + stmt, err := s.p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err) + sctx := MockContext() + builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) + domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) + plan, err := builder.Build(ctx, stmt) + testdata.OnRecord(func() { + output[i].Err = err != nil + }) + if output[i].Err { + require.Error(t, err) + continue + } + require.NoError(t, err, comment) + p := plan.(LogicalPlan) + p, err = logicalOptimize(ctx, builder.optFlag, p) + testdata.OnRecord(func() { + output[i].Best = ToString(p) + }) + require.NoError(t, err) + require.Equal(t, output[i].Best, ToString(p), comment) + } +} + +func TestTopNPushDown(t *testing.T) { + var input, output []string + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + s := createPlannerSuite() + ctx := context.TODO() + for i, tt := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, tt) + stmt, err := s.p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err) + sctx := MockContext() + builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) + domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) + p, err := builder.Build(ctx, stmt) + require.NoError(t, err) + p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i] = ToString(p) + }) + require.Equal(t, output[i], ToString(p), comment) + } +} + +func TestNameResolver(t *testing.T) { + tests := []struct { + sql string + err string + }{ + {"select a from t", ""}, + {"select c3 from t", "[planner:1054]Unknown column 'c3' in 'field list'"}, + {"select c1 from t4", "[schema:1146]Table 'test.t4' doesn't exist"}, + {"select * from t", ""}, + {"select t.* from t", ""}, + {"select t2.* from t", "[planner:1051]Unknown table 't2'"}, + {"select b as a, c as a from t group by a", "[planner:1052]Column 'a' in group statement is ambiguous"}, + {"select 1 as a, b as a, c as a from t group by a", ""}, + {"select a, b as a from t group by a+1", ""}, + {"select c, a as c from t order by c+1", ""}, + {"select * from t as t1, t as t2 join t as t3 on t2.a = t3.a", ""}, + {"select * from t as t1, t as t2 join t as t3 on t1.c1 = t2.a", "[planner:1054]Unknown column 't1.c1' in 'on clause'"}, + {"select a from t group by a having a = 3", ""}, + {"select a from t group by a having c2 = 3", "[planner:1054]Unknown column 'c2' in 'having clause'"}, + {"select a from t where exists (select b)", ""}, + {"select cnt from (select count(a) as cnt from t group by b) as t2 group by cnt", ""}, + {"select a from t where t11.a < t.a", "[planner:1054]Unknown column 't11.a' in 'where clause'"}, + {"select a from t having t11.c1 < t.a", "[planner:1054]Unknown column 't11.c1' in 'having clause'"}, + {"select a from t where t.a < t.a order by t11.c1", "[planner:1054]Unknown column 't11.c1' in 'order clause'"}, + {"select a from t group by t11.c1", "[planner:1054]Unknown column 't11.c1' in 'group statement'"}, + {"delete a from (select * from t ) as a, t", "[planner:1288]The target table a of the DELETE is not updatable"}, + {"delete b from (select * from t ) as a, t", "[planner:1109]Unknown table 'b' in MULTI DELETE"}, + {"select '' as fakeCol from t group by values(fakeCol)", "[planner:1054]Unknown column '' in 'VALUES() function'"}, + {"update t, (select * from t) as b set b.a = t.a", "[planner:1288]The target table b of the UPDATE is not updatable"}, + {"select row_number() over () from t group by 1", "[planner:1056]Can't group on 'row_number() over ()'"}, + {"select row_number() over () as x from t group by 1", "[planner:1056]Can't group on 'x'"}, + {"select sum(a) as x from t group by 1", "[planner:1056]Can't group on 'x'"}, + } + + s := createPlannerSuite() + ctx := context.Background() + for _, test := range tests { + comment := fmt.Sprintf("for %s", test.sql) + stmt, err := s.p.ParseOneStmt(test.sql, "", "") + require.NoError(t, err, comment) + s.ctx.GetSessionVars().SetHashJoinConcurrency(1) + + _, _, err = BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + if test.err == "" { + require.NoError(t, err) + } else { + require.EqualError(t, err, test.err) + } + } +} + +func TestOuterJoinEliminator(t *testing.T) { + var input, output []string + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + + s := createPlannerSuite() + ctx := context.TODO() + for i, tt := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, tt) + stmt, err := s.p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err) + sctx := MockContext() + builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) + domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) + p, err := builder.Build(ctx, stmt) + require.NoError(t, err) + p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) + require.NoError(t, err) + planString := ToString(p) + testdata.OnRecord(func() { + output[i] = planString + }) + require.Equal(t, output[i], planString, comment) + } +} + +func TestSelectView(t *testing.T) { + tests := []struct { + sql string + best string + }{ + { + sql: "select * from v", + best: "DataScan(t)->Projection", + }, + { + sql: "select v.b, v.c, v.d from v", + best: "DataScan(t)->Projection", + }, + } + s := createPlannerSuite() + ctx := context.TODO() + for i, tt := range tests { + comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) + stmt, err := s.p.ParseOneStmt(tt.sql, "", "") + require.NoError(t, err, comment) + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err) + builder, _ := NewPlanBuilder().Init(MockContext(), s.is, &hint.BlockHintProcessor{}) + p, err := builder.Build(ctx, stmt) + require.NoError(t, err) + p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) + require.NoError(t, err) + require.Equal(t, tt.best, ToString(p), comment) + } +} + +type plannerSuiteWithOptimizeVars struct { + *plannerSuite + optimizeVars map[string]string +} + +func TestWindowFunction(t *testing.T) { + s := new(plannerSuiteWithOptimizeVars) + s.plannerSuite = createPlannerSuite() + + s.optimizeVars = map[string]string{ + variable.TiDBWindowConcurrency: "1", + variable.TiDBCostModelVersion: "1", + } + defer func() { + s.optimizeVars = nil + }() + var input, output []string + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + s.doTestWindowFunction(t, input, output) +} + +func TestWindowParallelFunction(t *testing.T) { + s := new(plannerSuiteWithOptimizeVars) + s.plannerSuite = createPlannerSuite() + + s.optimizeVars = map[string]string{ + variable.TiDBWindowConcurrency: "4", + variable.TiDBCostModelVersion: "1", + } + defer func() { + s.optimizeVars = nil + }() + var input, output []string + planSuiteUnexportedData.LoadTestCases(t, &input, &output) + s.doTestWindowFunction(t, input, output) +} + +func (s *plannerSuiteWithOptimizeVars) doTestWindowFunction(t *testing.T, input, output []string) { + ctx := context.TODO() + for i, tt := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, tt) + p, stmt, err := s.optimize(ctx, tt) + if err != nil { + testdata.OnRecord(func() { + output[i] = err.Error() + }) + require.EqualError(t, err, output[i], comment) + continue + } + testdata.OnRecord(func() { + output[i] = ToString(p) + }) + require.Equal(t, output[i], ToString(p), comment) + + var sb strings.Builder + // After restore, the result should be the same. + err = stmt.Restore(format.NewRestoreCtx(format.DefaultRestoreFlags, &sb)) + require.NoError(t, err) + p, _, err = s.optimize(ctx, sb.String()) + if err != nil { + require.EqualError(t, err, output[i], comment) + continue + } + require.Equal(t, output[i], ToString(p), comment) + } +} + +func (s *plannerSuiteWithOptimizeVars) optimize(ctx context.Context, sql string) (PhysicalPlan, ast.Node, error) { + stmt, err := s.p.ParseOneStmt(sql, "", "") + if err != nil { + return nil, nil, err + } + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + if err != nil { + return nil, nil, err + } + + sctx := MockContext() + for k, v := range s.optimizeVars { + if err = sctx.GetSessionVars().SetSystemVar(k, v); err != nil { + return nil, nil, err + } + } + builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) + domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) + p, err := builder.Build(ctx, stmt) + if err != nil { + return nil, nil, err + } + p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) + if err != nil { + return nil, nil, err + } + p, _, err = physicalOptimize(p.(LogicalPlan), &PlanCounterDisabled) + return p.(PhysicalPlan), stmt, err +} + +func byItemsToProperty(byItems []*util.ByItems) *property.PhysicalProperty { + pp := &property.PhysicalProperty{} + for _, item := range byItems { + pp.SortItems = append(pp.SortItems, property.SortItem{Col: item.Expr.(*expression.Column), Desc: item.Desc}) + } + return pp +} + +func pathsName(paths []*candidatePath) string { + var names []string + for _, path := range paths { + if path.path.IsTablePath() { + names = append(names, "PRIMARY_KEY") + } else { + names = append(names, path.path.Index.Name.O) + } + } + return strings.Join(names, ",") +} + +func TestSkylinePruning(t *testing.T) { + tests := []struct { + sql string + result string + }{ + { + sql: "select * from t", + result: "PRIMARY_KEY", + }, + { + sql: "select * from t order by f", + result: "PRIMARY_KEY,f,f_g", + }, + { + sql: "select * from t where a > 1", + result: "PRIMARY_KEY", + }, + { + sql: "select * from t where a > 1 order by f", + result: "PRIMARY_KEY,f,f_g", + }, + { + sql: "select * from t where f > 1", + result: "PRIMARY_KEY,f,f_g", + }, + { + sql: "select f from t where f > 1", + result: "f,f_g", + }, + { + sql: "select f from t where f > 1 order by a", + result: "PRIMARY_KEY,f,f_g", + }, + { + sql: "select * from t where f > 1 and g > 1", + result: "PRIMARY_KEY,g,f_g", + }, + { + sql: "select count(1) from t", + result: "PRIMARY_KEY,c_d_e,f,g,f_g,c_d_e_str,e_d_c_str_prefix", + }, + { + sql: "select * from t where f > 3 and g = 5", + result: "PRIMARY_KEY,g,f_g", + }, + { + sql: "select * from t where g = 5 order by f", + result: "PRIMARY_KEY,g,f_g", + }, + { + sql: "select * from t where d = 3 order by c, e", + result: "PRIMARY_KEY,c_d_e", + }, + { + sql: "select * from t where d = 1 and f > 1 and g > 1 order by c, e", + result: "PRIMARY_KEY,c_d_e,g,f_g", + }, + } + s := createPlannerSuite() + ctx := context.TODO() + for i, tt := range tests { + comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) + stmt, err := s.p.ParseOneStmt(tt.sql, "", "") + require.NoError(t, err, comment) + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err) + sctx := MockContext() + builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) + domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) + p, err := builder.Build(ctx, stmt) + if err != nil { + require.EqualError(t, err, tt.result, comment) + continue + } + require.NoError(t, err, comment) + p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) + require.NoError(t, err, comment) + lp := p.(LogicalPlan) + _, err = lp.recursiveDeriveStats(nil) + require.NoError(t, err, comment) + var ds *DataSource + var byItems []*util.ByItems + for ds == nil { + switch v := lp.(type) { + case *DataSource: + ds = v + case *LogicalSort: + byItems = v.ByItems + lp = lp.Children()[0] + case *LogicalProjection: + newItems := make([]*util.ByItems, 0, len(byItems)) + for _, col := range byItems { + idx := v.schema.ColumnIndex(col.Expr.(*expression.Column)) + switch expr := v.Exprs[idx].(type) { + case *expression.Column: + newItems = append(newItems, &util.ByItems{Expr: expr, Desc: col.Desc}) + } + } + byItems = newItems + lp = lp.Children()[0] + default: + lp = lp.Children()[0] + } + } + paths := ds.skylinePruning(byItemsToProperty(byItems)) + require.Equal(t, tt.result, pathsName(paths), comment) + } +} + +func TestFastPlanContextTables(t *testing.T) { + tests := []struct { + sql string + fastPlan bool + }{ + { + "select * from t where a=1", + true, + }, + { + + "update t set f=0 where a=43215", + true, + }, + { + "delete from t where a =43215", + true, + }, + { + "select * from t where a>1", + false, + }, + } + s := createPlannerSuite() + s.ctx.GetSessionVars().SnapshotInfoschema = s.is + for _, tt := range tests { + stmt, err := s.p.ParseOneStmt(tt.sql, "", "") + require.NoError(t, err) + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err) + s.ctx.GetSessionVars().StmtCtx.Tables = nil + p := TryFastPlan(s.ctx, stmt) + if tt.fastPlan { + require.NotNil(t, p) + require.Equal(t, 1, len(s.ctx.GetSessionVars().StmtCtx.Tables)) + require.Equal(t, "t", s.ctx.GetSessionVars().StmtCtx.Tables[0].Table) + require.Equal(t, "test", s.ctx.GetSessionVars().StmtCtx.Tables[0].DB) + } else { + require.Nil(t, p) + require.Equal(t, 0, len(s.ctx.GetSessionVars().StmtCtx.Tables)) + } + } +} + +func TestUpdateEQCond(t *testing.T) { + tests := []struct { + sql string + best string + }{ + { + sql: "select t1.a from t t1, t t2 where t1.a = t2.a+1", + best: "Join{DataScan(t1)->DataScan(t2)->Projection}(test.t.a,Column#25)->Projection->Projection", + }, + } + s := createPlannerSuite() + ctx := context.TODO() + for i, tt := range tests { + comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) + stmt, err := s.p.ParseOneStmt(tt.sql, "", "") + require.NoError(t, err, comment) + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err) + sctx := MockContext() + builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) + domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) + p, err := builder.Build(ctx, stmt) + require.NoError(t, err) + p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) + require.NoError(t, err) + require.Equal(t, tt.best, ToString(p), comment) + } +} + +func TestConflictedJoinTypeHints(t *testing.T) { + sql := "select /*+ INL_JOIN(t1) HASH_JOIN(t1) */ * from t t1, t t2 where t1.e = t2.e" + s := createPlannerSuite() + ctx := context.TODO() + stmt, err := s.p.ParseOneStmt(sql, "", "") + require.NoError(t, err) + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err) + sctx := MockContext() + builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) + domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) + p, err := builder.Build(ctx, stmt) + require.NoError(t, err) + p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) + require.NoError(t, err) + proj, ok := p.(*LogicalProjection) + require.True(t, ok) + join, ok := proj.Children()[0].(*LogicalJoin) + require.True(t, ok) + require.Nil(t, join.hintInfo) + require.Equal(t, uint(0), join.preferJoinType) +} + +func TestSimplyOuterJoinWithOnlyOuterExpr(t *testing.T) { + s := createPlannerSuite() + sql := "select * from t t1 right join t t0 ON TRUE where CONCAT_WS(t0.e=t0.e, 0, NULL) IS NULL" + ctx := context.TODO() + stmt, err := s.p.ParseOneStmt(sql, "", "") + require.NoError(t, err) + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err) + sctx := MockContext() + builder, _ := NewPlanBuilder().Init(sctx, s.is, &hint.BlockHintProcessor{}) + domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(s.is) + p, err := builder.Build(ctx, stmt) + require.NoError(t, err) + p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) + require.NoError(t, err) + proj, ok := p.(*LogicalProjection) + require.True(t, ok) + join, ok := proj.Children()[0].(*LogicalJoin) + require.True(t, ok) + // previous wrong JoinType is InnerJoin + require.Equal(t, RightOuterJoin, join.JoinType) +} + +func TestResolvingCorrelatedAggregate(t *testing.T) { + tests := []struct { + sql string + best string + }{ + { + sql: "select (select count(a)) from t", + best: "Apply{DataScan(t)->Aggr(count(test.t.a))->Dual->Projection->MaxOneRow}->Projection", + }, + { + sql: "select (select count(n.a) from t) from t n", + best: "Apply{DataScan(n)->Aggr(count(test.t.a))->DataScan(t)->Projection->MaxOneRow}->Projection", + }, + { + sql: "select (select sum(count(a))) from t", + best: "Apply{DataScan(t)->Aggr(count(test.t.a))->Dual->Aggr(sum(Column#13))->MaxOneRow}->Projection", + }, + { + sql: "select (select sum(count(n.a)) from t) from t n", + best: "Apply{DataScan(n)->Aggr(count(test.t.a))->DataScan(t)->Aggr(sum(Column#25))->MaxOneRow}->Projection", + }, + { + sql: "select (select cnt from (select count(a) as cnt) n) from t", + best: "Apply{DataScan(t)->Aggr(count(test.t.a))->Dual->Projection->MaxOneRow}->Projection", + }, + { + sql: "select sum(a), sum(a), count(a), (select count(a)) from t", + best: "Apply{DataScan(t)->Aggr(sum(test.t.a),count(test.t.a))->Dual->Projection->MaxOneRow}->Projection", + }, + } + + s := createPlannerSuite() + ctx := context.TODO() + for i, tt := range tests { + comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) + stmt, err := s.p.ParseOneStmt(tt.sql, "", "") + require.NoError(t, err, comment) + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err, comment) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err, comment) + p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagEliminateProjection|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) + require.NoError(t, err, comment) + require.Equal(t, tt.best, ToString(p), comment) + } +} + +func TestFastPathInvalidBatchPointGet(t *testing.T) { + // #22040 + tt := []struct { + sql string + fastPlan bool + }{ + { + // column count doesn't match, not use idx + sql: "select * from t where (a,b) in ((1,2),1)", + fastPlan: false, + }, + { + // column count doesn't match, not use idx + sql: "select * from t where (a,b) in (1,2)", + fastPlan: false, + }, + { + // column count doesn't match, use idx + sql: "select * from t where (f,g) in ((1,2),1)", + fastPlan: false, + }, + { + // column count doesn't match, use idx + sql: "select * from t where (f,g) in (1,2)", + fastPlan: false, + }, + } + s := createPlannerSuite() + for i, tc := range tt { + comment := fmt.Sprintf("case:%v sql:%s", i, tc.sql) + stmt, err := s.p.ParseOneStmt(tc.sql, "", "") + require.NoError(t, err, comment) + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err, comment) + plan := TryFastPlan(s.ctx, stmt) + if tc.fastPlan { + require.NotNil(t, plan) + } else { + require.Nil(t, plan) + } + } +} + +func TestTraceFastPlan(t *testing.T) { + s := createPlannerSuite() + s.ctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = true + defer func() { + s.ctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = false + }() + s.ctx.GetSessionVars().SnapshotInfoschema = s.is + sql := "select * from t where a=1" + comment := fmt.Sprintf("sql:%s", sql) + stmt, err := s.p.ParseOneStmt(sql, "", "") + require.NoError(t, err, comment) + err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + require.NoError(t, err, comment) + plan := TryFastPlan(s.ctx, stmt) + require.NotNil(t, plan) + require.NotNil(t, s.ctx.GetSessionVars().StmtCtx.OptimizeTracer) + require.NotNil(t, s.ctx.GetSessionVars().StmtCtx.OptimizeTracer.FinalPlan) + require.True(t, s.ctx.GetSessionVars().StmtCtx.OptimizeTracer.IsFastPlan) +} + +func TestWindowLogicalPlanAmbiguous(t *testing.T) { + sql := "select a, max(a) over(), sum(a) over() from t" + var planString string + // The ambiguous logical plan which contains window function can usually be found in 100 iterations. + iterations := 100 + s := createPlannerSuite() + for i := 0; i < iterations; i++ { + stmt, err := s.p.ParseOneStmt(sql, "", "") + require.NoError(t, err) + p, _, err := BuildLogicalPlanForTest(context.Background(), s.ctx, stmt, s.is) + require.NoError(t, err) + if planString == "" { + planString = ToString(p) + } else { + require.Equal(t, ToString(p), planString) + } + } +} + +func TestRemoveOrderbyInSubquery(t *testing.T) { + tests := []struct { + sql string + best string + }{ + { + sql: "select * from t order by a", + best: "DataScan(t)->Projection->Sort", + }, + { + sql: "select (select 1) from t order by a", + best: "DataScan(t)->Projection->Sort->Projection", + }, + { + sql: "select count(*) from (select b from t order by a) n", + best: "DataScan(t)->Projection->Projection->Aggr(count(1),firstrow(test.t.b))->Projection", + }, + { + sql: "select count(1) from (select b from t order by a limit 1) n", + best: "DataScan(t)->Projection->Sort->Limit->Projection->Aggr(count(1),firstrow(test.t.b))->Projection", + }, + } + + s := createPlannerSuite() + s.ctx.GetSessionVars().RemoveOrderbyInSubquery = true + ctx := context.TODO() + for i, tt := range tests { + comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) + stmt, err := s.p.ParseOneStmt(tt.sql, "", "") + require.NoError(t, err, comment) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) + require.NoError(t, err, comment) + require.Equal(t, tt.best, ToString(p), comment) } } diff --git a/planner/core/main_test.go b/planner/core/main_test.go index efe59fd011686..b9cbcdc9e44bd 100644 --- a/planner/core/main_test.go +++ b/planner/core/main_test.go @@ -18,15 +18,24 @@ import ( "flag" "testing" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/testkit/testmain" "github.com/pingcap/tidb/testkit/testsetup" "go.uber.org/goleak" ) +var testDataMap = make(testdata.BookKeeper) +var planSuiteUnexportedData testdata.TestData +var indexMergeSuiteData testdata.TestData + func TestMain(m *testing.M) { testsetup.SetupForCommonTest() flag.Parse() - + testDataMap.LoadTestSuiteData("testdata", "plan_suite_unexported") + testDataMap.LoadTestSuiteData("testdata", "index_merge_suite") + indexMergeSuiteData = testDataMap["index_merge_suite"] + planSuiteUnexportedData = testDataMap["plan_suite_unexported"] opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), @@ -36,5 +45,14 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - goleak.VerifyTestMain(m, opts...) + callback := func(i int) int { + testDataMap.GenerateOutputIfNeeded() + return i + } + + goleak.VerifyTestMain(testmain.WrapTestingM(m, callback), opts...) +} + +func GetIndexMergeSuiteData() testdata.TestData { + return testDataMap["index_merge_suite"] } diff --git a/planner/core/casetest/testdata/plan_suite_unexported_in.json b/planner/core/testdata/plan_suite_unexported_in.json similarity index 100% rename from planner/core/casetest/testdata/plan_suite_unexported_in.json rename to planner/core/testdata/plan_suite_unexported_in.json diff --git a/planner/core/casetest/testdata/plan_suite_unexported_out.json b/planner/core/testdata/plan_suite_unexported_out.json similarity index 100% rename from planner/core/casetest/testdata/plan_suite_unexported_out.json rename to planner/core/testdata/plan_suite_unexported_out.json From 2e467f6e7f906541bec79e336becf29b5cf049f7 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 15 Feb 2023 00:30:50 +0800 Subject: [PATCH 03/12] *: improve planner/core to make test faster Signed-off-by: Weizhen Wang --- planner/core/internal/testkit.go | 1 + 1 file changed, 1 insertion(+) diff --git a/planner/core/internal/testkit.go b/planner/core/internal/testkit.go index ed335ab783466..6c3e7f6b022f1 100644 --- a/planner/core/internal/testkit.go +++ b/planner/core/internal/testkit.go @@ -24,6 +24,7 @@ import ( "github.com/stretchr/testify/require" ) +// SetTiFlashReplica is to set TiFlash replica func SetTiFlashReplica(t *testing.T, dom *domain.Domain, dbName, tableName string) { is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr(dbName)) From 68ad17e8a7d9d03205990db8c56250284395113c Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 15 Feb 2023 00:31:29 +0800 Subject: [PATCH 04/12] *: improve planner/core to make test faster Signed-off-by: Weizhen Wang --- planner/core/collect_column_stats_usage_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/collect_column_stats_usage_test.go b/planner/core/collect_column_stats_usage_test.go index 82f40a834bf59..249f210050c56 100644 --- a/planner/core/collect_column_stats_usage_test.go +++ b/planner/core/collect_column_stats_usage_test.go @@ -324,7 +324,7 @@ func TestCollectHistNeededColumns(t *testing.T) { }, } - s := casetest.createPlannerSuite() + s := createPlannerSuite() ctx := context.Background() for _, tt := range tests { comment := fmt.Sprintf("sql: %s", tt.sql) From 2bb8e044af791dda9c1d2232c01ad82b9ff616af Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 15 Feb 2023 00:43:21 +0800 Subject: [PATCH 05/12] *: improve planner/core to make test faster Signed-off-by: Weizhen Wang --- planner/core/casetest/BUILD.bazel | 5 + planner/core/casetest/cbo_test.go | 633 +++++++++++++++++++++++++++++ planner/core/casetest/main_test.go | 2 +- planner/core/cbo_test.go | 605 --------------------------- 4 files changed, 639 insertions(+), 606 deletions(-) create mode 100644 planner/core/casetest/cbo_test.go diff --git a/planner/core/casetest/BUILD.bazel b/planner/core/casetest/BUILD.bazel index 7f9196de4bd1b..7604ca5be0c88 100644 --- a/planner/core/casetest/BUILD.bazel +++ b/planner/core/casetest/BUILD.bazel @@ -5,6 +5,7 @@ go_test( timeout = "short", srcs = [ "binary_plan_test.go", + "cbo_test.go", "expression_rewriter_test.go", "main_test.go", "plan_test.go", @@ -23,6 +24,7 @@ go_test( deps = [ "//config", "//domain", + "//executor", "//expression", "//expression/aggregation", "//infoschema", @@ -34,8 +36,11 @@ go_test( "//planner/core", "//planner/core/internal", "//planner/property", + "//session", "//sessionctx/variable", "//sessiontxn", + "//statistics", + "//statistics/handle", "//testkit", "//testkit/testdata", "//testkit/testmain", diff --git a/planner/core/casetest/cbo_test.go b/planner/core/casetest/cbo_test.go new file mode 100644 index 0000000000000..1cae683cefd94 --- /dev/null +++ b/planner/core/casetest/cbo_test.go @@ -0,0 +1,633 @@ +// 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 ( + "context" + "encoding/json" + "fmt" + "os" + "path/filepath" + "testing" + + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/planner" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/stretchr/testify/require" +) + +func loadTableStats(fileName string, dom *domain.Domain) error { + statsPath := filepath.Join("testdata", fileName) + bytes, err := os.ReadFile(statsPath) + if err != nil { + return err + } + statsTbl := &handle.JSONTable{} + err = json.Unmarshal(bytes, statsTbl) + if err != nil { + return err + } + statsHandle := dom.StatsHandle() + err = statsHandle.LoadStatsFromJSON(dom.InfoSchema(), statsTbl) + if err != nil { + return err + } + return nil +} + +// TestCBOWithoutAnalyze tests the plan with stats that only have count info. +func TestCBOWithoutAnalyze(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("create table t1 (a int)") + testKit.MustExec("create table t2 (a int)") + h := dom.StatsHandle() + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + testKit.MustExec("insert into t1 values (1), (2), (3), (4), (5), (6)") + testKit.MustExec("insert into t2 values (1), (2), (3), (4), (5), (6)") + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(dom.InfoSchema())) + var input []string + var output []struct { + SQL string + Plan []string + } + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + for i, sql := range input { + plan := testKit.MustQuery(sql) + testdata.OnRecord(func() { + output[i].SQL = sql + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) + }) + plan.Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestStraightJoin(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + h := dom.StatsHandle() + for _, tblName := range []string{"t1", "t2", "t3", "t4"} { + testKit.MustExec(fmt.Sprintf("create table %s (a int)", tblName)) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + } + var input []string + var output [][]string + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) + }) + testKit.MustQuery(tt).Check(testkit.Rows(output[i]...)) + } +} + +func TestTableDual(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + + testKit := testkit.NewTestKit(t, store) + testKit.MustExec(`use test`) + h := dom.StatsHandle() + testKit.MustExec(`create table t(a int)`) + testKit.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)") + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(dom.InfoSchema())) + var input []string + var output []struct { + SQL string + Plan []string + } + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + for i, sql := range input { + plan := testKit.MustQuery(sql) + testdata.OnRecord(func() { + output[i].SQL = sql + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) + }) + plan.Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestEstimation(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + statistics.RatioOfPseudoEstimate.Store(10.0) + defer statistics.RatioOfPseudoEstimate.Store(0.7) + testKit.MustExec("use test") + testKit.MustExec("set tidb_cost_model_version=2") + testKit.MustExec("create table t (a int)") + testKit.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)") + testKit.MustExec("insert into t select * from t") + testKit.MustExec("insert into t select * from t") + h := dom.StatsHandle() + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + testKit.MustExec("analyze table t") + for i := 1; i <= 8; i++ { + testKit.MustExec("delete from t where a = ?", i) + } + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(dom.InfoSchema())) + var input []string + var output []struct { + SQL string + Plan []string + } + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + for i, sql := range input { + plan := testKit.MustQuery(sql) + testdata.OnRecord(func() { + output[i].SQL = sql + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) + }) + plan.Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestIndexRead(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("set tidb_cost_model_version=2") + testKit.MustExec("set @@session.tidb_executor_concurrency = 4;") + testKit.MustExec("set @@session.tidb_hash_join_concurrency = 5;") + testKit.MustExec("set @@session.tidb_distsql_scan_concurrency = 15;") + + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t, t1") + testKit.MustExec("create table t (a int primary key, b int, c varchar(200), d datetime DEFAULT CURRENT_TIMESTAMP, e int, ts timestamp DEFAULT CURRENT_TIMESTAMP)") + testKit.MustExec("create index b on t (b)") + testKit.MustExec("create index d on t (d)") + testKit.MustExec("create index e on t (e)") + testKit.MustExec("create index b_c on t (b,c)") + testKit.MustExec("create index ts on t (ts)") + testKit.MustExec("create table t1 (a int, b int, index idx(a), index idxx(b))") + + // Default RPC encoding may cause statistics explain result differ and then the test unstable. + testKit.MustExec("set @@tidb_enable_chunk_rpc = on") + + // This stats is generated by following format: + // fill (a, b, c, e) as (i*100+j, i, i+j, i*100+j), i and j is dependent and range of this two are [0, 99]. + require.NoError(t, loadTableStats("analyzesSuiteTestIndexReadT.json", dom)) + for i := 1; i < 16; i++ { + testKit.MustExec(fmt.Sprintf("insert into t1 values(%v, %v)", i, i)) + } + testKit.MustExec("analyze table t1") + ctx := testKit.Session() + var input, output []string + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + + for i, tt := range input { + stmts, err := session.Parse(ctx, tt) + require.NoError(t, err) + require.Len(t, stmts, 1) + stmt := stmts[0] + ret := &core.PreprocessorReturn{} + err = core.Preprocess(context.Background(), ctx, stmt, core.WithPreprocessorReturn(ret)) + require.NoError(t, err) + p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) + require.NoError(t, err) + planString := core.ToString(p) + testdata.OnRecord(func() { + output[i] = planString + }) + require.Equalf(t, output[i], planString, "case: %v", tt) + } +} + +func TestEmptyTable(t *testing.T) { + store := testkit.CreateMockStore(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("set tidb_cost_model_version=2") + testKit.MustExec("drop table if exists t, t1") + testKit.MustExec("create table t (c1 int)") + testKit.MustExec("create table t1 (c1 int)") + testKit.MustExec("analyze table t, t1") + var input, output []string + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + ctx := testKit.Session() + stmts, err := session.Parse(ctx, tt) + require.NoError(t, err) + require.Len(t, stmts, 1) + stmt := stmts[0] + ret := &core.PreprocessorReturn{} + err = core.Preprocess(context.Background(), ctx, stmt, core.WithPreprocessorReturn(ret)) + require.NoError(t, err) + p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) + require.NoError(t, err) + planString := core.ToString(p) + testdata.OnRecord(func() { + output[i] = planString + }) + require.Equalf(t, output[i], planString, "case: %v", tt) + } +} + +func TestAnalyze(t *testing.T) { + store := testkit.CreateMockStore(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t, t1, t2, t3") + testKit.MustExec("create table t (a int, b int)") + testKit.MustExec("create index a on t (a)") + testKit.MustExec("create index b on t (b)") + testKit.MustExec("insert into t (a,b) values (1,1),(1,2),(1,3),(1,4),(2,5),(2,6),(2,7),(2,8)") + testKit.MustExec("analyze table t") + + testKit.MustExec("create table t1 (a int, b int)") + testKit.MustExec("create index a on t1 (a)") + testKit.MustExec("create index b on t1 (b)") + testKit.MustExec("insert into t1 (a,b) values (1,1),(1,2),(1,3),(1,4),(2,5),(2,6),(2,7),(2,8)") + + testKit.MustExec("create table t2 (a int, b int)") + testKit.MustExec("create index a on t2 (a)") + testKit.MustExec("create index b on t2 (b)") + testKit.MustExec("insert into t2 (a,b) values (1,1),(1,2),(1,3),(1,4),(2,5),(2,6),(2,7),(2,8)") + testKit.MustExec("analyze table t2 index a") + + testKit.MustExec("create table t3 (a int, b int)") + testKit.MustExec("create index a on t3 (a)") + + testKit.MustExec("set @@tidb_partition_prune_mode = 'static';") + testKit.MustExec("create table t4 (a int, b int) partition by range (a) (partition p1 values less than (2), partition p2 values less than (3))") + testKit.MustExec("create index a on t4 (a)") + testKit.MustExec("create index b on t4 (b)") + testKit.MustExec("insert into t4 (a,b) values (1,1),(1,2),(1,3),(1,4),(2,5),(2,6),(2,7),(2,8)") + testKit.MustExec("analyze table t4") + + testKit.MustExec("create view v as select * from t") + _, err := testKit.Exec("analyze table v") + require.EqualError(t, err, "analyze view v is not supported now") + testKit.MustExec("drop view v") + + testKit.MustExec("create sequence seq") + _, err = testKit.Exec("analyze table seq") + require.EqualError(t, err, "analyze sequence seq is not supported now") + testKit.MustExec("drop sequence seq") + + var input, output []string + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + + for i, tt := range input { + ctx := testKit.Session() + stmts, err := session.Parse(ctx, tt) + require.NoError(t, err) + require.Len(t, stmts, 1) + stmt := stmts[0] + err = executor.ResetContextOfStmt(ctx, stmt) + require.NoError(t, err) + ret := &core.PreprocessorReturn{} + err = core.Preprocess(context.Background(), ctx, stmt, core.WithPreprocessorReturn(ret)) + require.NoError(t, err) + p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) + require.NoError(t, err) + planString := core.ToString(p) + testdata.OnRecord(func() { + output[i] = planString + }) + require.Equalf(t, output[i], planString, "case: %v", tt) + } +} + +func TestOutdatedAnalyze(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("create table t (a int, b int, index idx(a))") + for i := 0; i < 10; i++ { + testKit.MustExec(fmt.Sprintf("insert into t values (%d,%d)", i, i)) + } + h := dom.StatsHandle() + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + testKit.MustExec("analyze table t") + testKit.MustExec("insert into t select * from t") + testKit.MustExec("insert into t select * from t") + testKit.MustExec("insert into t select * from t") + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(dom.InfoSchema())) + var input []struct { + SQL string + EnablePseudoForOutdatedStats bool + RatioOfPseudoEstimate float64 + } + var output []struct { + SQL string + EnablePseudoForOutdatedStats bool + RatioOfPseudoEstimate float64 + Plan []string + } + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testKit.Session().GetSessionVars().SetEnablePseudoForOutdatedStats(tt.EnablePseudoForOutdatedStats) + statistics.RatioOfPseudoEstimate.Store(tt.RatioOfPseudoEstimate) + plan := testKit.MustQuery(tt.SQL) + testdata.OnRecord(func() { + output[i].SQL = tt.SQL + output[i].EnablePseudoForOutdatedStats = tt.EnablePseudoForOutdatedStats + output[i].RatioOfPseudoEstimate = tt.RatioOfPseudoEstimate + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) + }) + plan.Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestNullCount(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t (a int, b int, index idx(a))") + testKit.MustExec("insert into t values (null, null), (null, null)") + testKit.MustExec("analyze table t") + var input []string + var output [][]string + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + for i := 0; i < 2; i++ { + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + }) + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) + } + h := dom.StatsHandle() + h.Clear() + require.NoError(t, h.Update(dom.InfoSchema())) + for i := 2; i < 4; i++ { + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + }) + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) + } +} + +func TestCorrelatedEstimation(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 sql_mode='STRICT_TRANS_TABLES'") // disable only full group by + tk.MustExec("create table t(a int, b int, c int, index idx(c,b,a))") + tk.MustExec("insert into t values(1,1,1), (2,2,2), (3,3,3), (4,4,4), (5,5,5), (6,6,6), (7,7,7), (8,8,8), (9,9,9),(10,10,10)") + tk.MustExec("analyze table t") + var ( + input []string + output [][]string + ) + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + rs := tk.MustQuery(tt) + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(rs.Rows()) + }) + rs.Check(testkit.Rows(output[i]...)) + } +} + +func TestInconsistentEstimation(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int, c int, index ab(a,b), index ac(a,c))") + tk.MustExec("insert into t values (1,1,1), (1000,1000,1000)") + for i := 0; i < 10; i++ { + tk.MustExec("insert into t values (5,5,5), (10,10,10)") + } + tk.MustExec("set @@tidb_analyze_version=1") + tk.MustExec("analyze table t with 2 buckets") + // Force using the histogram to estimate. + tk.MustExec("update mysql.stats_histograms set stats_ver = 0") + dom.StatsHandle().Clear() + require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema())) + var input []string + var output []struct { + SQL string + Plan []string + } + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + for i, sql := range input { + plan := tk.MustQuery(sql) + testdata.OnRecord(func() { + output[i].SQL = sql + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) + }) + plan.Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestIssue9562(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + var input [][]string + var output []struct { + SQL []string + Plan []string + } + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + for i, ts := range input { + for j, tt := range ts { + if j != len(ts)-1 { + tk.MustExec(tt) + } + testdata.OnRecord(func() { + output[i].SQL = ts + if j == len(ts)-1 { + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + } + }) + if j == len(ts)-1 { + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } + } + } +} + +func TestLimitCrossEstimation(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("set @@session.tidb_executor_concurrency = 4;") + tk.MustExec("set @@session.tidb_hash_join_concurrency = 5;") + tk.MustExec("set @@session.tidb_distsql_scan_concurrency = 15;") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int primary key, b int not null, c int not null default 0, index idx_bc(b, c))") + var input [][]string + var output []struct { + SQL []string + Plan []string + } + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + for i, ts := range input { + for j, tt := range ts { + if j != len(ts)-1 { + tk.MustExec(tt) + } + testdata.OnRecord(func() { + output[i].SQL = ts + if j == len(ts)-1 { + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + } + }) + if j == len(ts)-1 { + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } + } + } +} + +func TestLowSelIndexGreedySearch(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("set tidb_cost_model_version=2") + testKit.MustExec(`set tidb_opt_limit_push_down_threshold=0`) + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t (a varchar(32) default null, b varchar(10) default null, c varchar(12) default null, d varchar(32) default null, e bigint(10) default null, key idx1 (d,a), key idx2 (a,c), key idx3 (c,b), key idx4 (e))") + require.NoError(t, loadTableStats("analyzeSuiteTestLowSelIndexGreedySearchT.json", dom)) + var input []string + var output []struct { + SQL string + Plan []string + } + // The test purposes are: + // - index `idx2` runs much faster than `idx4` experimentally; + // - estimated row count of IndexLookUp should be 0; + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) + }) + testKit.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestTiFlashCostModel(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("create table t (a int, b int, c int, primary key(a))") + tk.MustExec("insert into t values(1,1,1), (2,2,2), (3,3,3)") + + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + var input, output [][]string + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + for i, ts := range input { + for j, tt := range ts { + if j != len(ts)-1 { + tk.MustExec(tt) + } + testdata.OnRecord(func() { + if j == len(ts)-1 { + output[i] = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + } + }) + if j == len(ts)-1 { + tk.MustQuery(tt).Check(testkit.Rows(output[i]...)) + } + } + } +} + +func TestIndexEqualUnknown(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t, t1") + testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + testKit.MustExec("CREATE TABLE t(a bigint(20) NOT NULL, b bigint(20) NOT NULL, c bigint(20) NOT NULL, PRIMARY KEY (a,c,b), KEY (b))") + require.NoError(t, loadTableStats("analyzeSuiteTestIndexEqualUnknownT.json", dom)) + var input []string + var output []struct { + SQL string + Plan []string + } + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) + }) + testKit.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestLimitIndexEstimation(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("drop table if exists t") + tk.MustExec("create table t(a int, b int, key idx_a(a), key idx_b(b))") + tk.MustExec("set session tidb_enable_extended_stats = on") + // Values in column a are from 1 to 1000000, values in column b are from 1000000 to 1, + // these 2 columns are strictly correlated in reverse order. + require.NoError(t, loadTableStats("analyzeSuiteTestLimitIndexEstimationT.json", dom)) + var input []string + var output []struct { + SQL string + Plan []string + } + + analyzeSuiteData := GetAnalyzeSuiteData() + analyzeSuiteData.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...)) + } +} diff --git a/planner/core/casetest/main_test.go b/planner/core/casetest/main_test.go index 8f4c1a1351f0e..c725e75230503 100644 --- a/planner/core/casetest/main_test.go +++ b/planner/core/casetest/main_test.go @@ -1,4 +1,4 @@ -// Copyright 2021 PingCAP, Inc. +// 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. diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 8448a66cd0bf0..4716a1f0798f5 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -16,47 +16,19 @@ package core_test import ( "context" - "encoding/json" "fmt" - "os" - "path/filepath" "strings" "testing" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/executor" - "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/planner" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" - "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/testkit/testdata" "github.com/stretchr/testify/require" ) -func loadTableStats(fileName string, dom *domain.Domain) error { - statsPath := filepath.Join("testdata", fileName) - bytes, err := os.ReadFile(statsPath) - if err != nil { - return err - } - statsTbl := &handle.JSONTable{} - err = json.Unmarshal(bytes, statsTbl) - if err != nil { - return err - } - statsHandle := dom.StatsHandle() - err = statsHandle.LoadStatsFromJSON(dom.InfoSchema(), statsTbl) - if err != nil { - return err - } - return nil -} - func TestExplainCostTrace(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -108,124 +80,6 @@ func TestExplainAnalyze(t *testing.T) { } } -// TestCBOWithoutAnalyze tests the plan with stats that only have count info. -func TestCBOWithoutAnalyze(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("create table t1 (a int)") - testKit.MustExec("create table t2 (a int)") - h := dom.StatsHandle() - require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) - require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) - testKit.MustExec("insert into t1 values (1), (2), (3), (4), (5), (6)") - testKit.MustExec("insert into t2 values (1), (2), (3), (4), (5), (6)") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.Update(dom.InfoSchema())) - var input []string - var output []struct { - SQL string - Plan []string - } - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - for i, sql := range input { - plan := testKit.MustQuery(sql) - testdata.OnRecord(func() { - output[i].SQL = sql - output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) - }) - plan.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestStraightJoin(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - h := dom.StatsHandle() - for _, tblName := range []string{"t1", "t2", "t3", "t4"} { - testKit.MustExec(fmt.Sprintf("create table %s (a int)", tblName)) - require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) - } - var input []string - var output [][]string - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) - }) - testKit.MustQuery(tt).Check(testkit.Rows(output[i]...)) - } -} - -func TestTableDual(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - - testKit := testkit.NewTestKit(t, store) - testKit.MustExec(`use test`) - h := dom.StatsHandle() - testKit.MustExec(`create table t(a int)`) - testKit.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)") - require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) - - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.Update(dom.InfoSchema())) - var input []string - var output []struct { - SQL string - Plan []string - } - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - for i, sql := range input { - plan := testKit.MustQuery(sql) - testdata.OnRecord(func() { - output[i].SQL = sql - output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) - }) - plan.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestEstimation(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - statistics.RatioOfPseudoEstimate.Store(10.0) - defer statistics.RatioOfPseudoEstimate.Store(0.7) - testKit.MustExec("use test") - testKit.MustExec("set tidb_cost_model_version=2") - testKit.MustExec("create table t (a int)") - testKit.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)") - testKit.MustExec("insert into t select * from t") - testKit.MustExec("insert into t select * from t") - h := dom.StatsHandle() - require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t") - for i := 1; i <= 8; i++ { - testKit.MustExec("delete from t where a = ?", i) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.Update(dom.InfoSchema())) - var input []string - var output []struct { - SQL string - Plan []string - } - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - for i, sql := range input { - plan := testKit.MustQuery(sql) - testdata.OnRecord(func() { - output[i].SQL = sql - output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) - }) - plan.Check(testkit.Rows(output[i].Plan...)) - } -} - func constructInsertSQL(i, n int) string { sql := "insert into t (a,b,c,e)values " for j := 0; j < n; j++ { @@ -237,284 +91,6 @@ func constructInsertSQL(i, n int) string { return sql } -func TestIndexRead(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("set tidb_cost_model_version=2") - testKit.MustExec("set @@session.tidb_executor_concurrency = 4;") - testKit.MustExec("set @@session.tidb_hash_join_concurrency = 5;") - testKit.MustExec("set @@session.tidb_distsql_scan_concurrency = 15;") - - testKit.MustExec("use test") - testKit.MustExec("drop table if exists t, t1") - testKit.MustExec("create table t (a int primary key, b int, c varchar(200), d datetime DEFAULT CURRENT_TIMESTAMP, e int, ts timestamp DEFAULT CURRENT_TIMESTAMP)") - testKit.MustExec("create index b on t (b)") - testKit.MustExec("create index d on t (d)") - testKit.MustExec("create index e on t (e)") - testKit.MustExec("create index b_c on t (b,c)") - testKit.MustExec("create index ts on t (ts)") - testKit.MustExec("create table t1 (a int, b int, index idx(a), index idxx(b))") - - // Default RPC encoding may cause statistics explain result differ and then the test unstable. - testKit.MustExec("set @@tidb_enable_chunk_rpc = on") - - // This stats is generated by following format: - // fill (a, b, c, e) as (i*100+j, i, i+j, i*100+j), i and j is dependent and range of this two are [0, 99]. - require.NoError(t, loadTableStats("analyzesSuiteTestIndexReadT.json", dom)) - for i := 1; i < 16; i++ { - testKit.MustExec(fmt.Sprintf("insert into t1 values(%v, %v)", i, i)) - } - testKit.MustExec("analyze table t1") - ctx := testKit.Session() - var input, output []string - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - - for i, tt := range input { - stmts, err := session.Parse(ctx, tt) - require.NoError(t, err) - require.Len(t, stmts, 1) - stmt := stmts[0] - ret := &core.PreprocessorReturn{} - err = core.Preprocess(context.Background(), ctx, stmt, core.WithPreprocessorReturn(ret)) - require.NoError(t, err) - p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) - require.NoError(t, err) - planString := core.ToString(p) - testdata.OnRecord(func() { - output[i] = planString - }) - require.Equalf(t, output[i], planString, "case: %v", tt) - } -} - -func TestEmptyTable(t *testing.T) { - store := testkit.CreateMockStore(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("set tidb_cost_model_version=2") - testKit.MustExec("drop table if exists t, t1") - testKit.MustExec("create table t (c1 int)") - testKit.MustExec("create table t1 (c1 int)") - testKit.MustExec("analyze table t, t1") - var input, output []string - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - ctx := testKit.Session() - stmts, err := session.Parse(ctx, tt) - require.NoError(t, err) - require.Len(t, stmts, 1) - stmt := stmts[0] - ret := &core.PreprocessorReturn{} - err = core.Preprocess(context.Background(), ctx, stmt, core.WithPreprocessorReturn(ret)) - require.NoError(t, err) - p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) - require.NoError(t, err) - planString := core.ToString(p) - testdata.OnRecord(func() { - output[i] = planString - }) - require.Equalf(t, output[i], planString, "case: %v", tt) - } -} - -func TestAnalyze(t *testing.T) { - store := testkit.CreateMockStore(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("drop table if exists t, t1, t2, t3") - testKit.MustExec("create table t (a int, b int)") - testKit.MustExec("create index a on t (a)") - testKit.MustExec("create index b on t (b)") - testKit.MustExec("insert into t (a,b) values (1,1),(1,2),(1,3),(1,4),(2,5),(2,6),(2,7),(2,8)") - testKit.MustExec("analyze table t") - - testKit.MustExec("create table t1 (a int, b int)") - testKit.MustExec("create index a on t1 (a)") - testKit.MustExec("create index b on t1 (b)") - testKit.MustExec("insert into t1 (a,b) values (1,1),(1,2),(1,3),(1,4),(2,5),(2,6),(2,7),(2,8)") - - testKit.MustExec("create table t2 (a int, b int)") - testKit.MustExec("create index a on t2 (a)") - testKit.MustExec("create index b on t2 (b)") - testKit.MustExec("insert into t2 (a,b) values (1,1),(1,2),(1,3),(1,4),(2,5),(2,6),(2,7),(2,8)") - testKit.MustExec("analyze table t2 index a") - - testKit.MustExec("create table t3 (a int, b int)") - testKit.MustExec("create index a on t3 (a)") - - testKit.MustExec("set @@tidb_partition_prune_mode = 'static';") - testKit.MustExec("create table t4 (a int, b int) partition by range (a) (partition p1 values less than (2), partition p2 values less than (3))") - testKit.MustExec("create index a on t4 (a)") - testKit.MustExec("create index b on t4 (b)") - testKit.MustExec("insert into t4 (a,b) values (1,1),(1,2),(1,3),(1,4),(2,5),(2,6),(2,7),(2,8)") - testKit.MustExec("analyze table t4") - - testKit.MustExec("create view v as select * from t") - _, err := testKit.Exec("analyze table v") - require.EqualError(t, err, "analyze view v is not supported now") - testKit.MustExec("drop view v") - - testKit.MustExec("create sequence seq") - _, err = testKit.Exec("analyze table seq") - require.EqualError(t, err, "analyze sequence seq is not supported now") - testKit.MustExec("drop sequence seq") - - var input, output []string - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - - for i, tt := range input { - ctx := testKit.Session() - stmts, err := session.Parse(ctx, tt) - require.NoError(t, err) - require.Len(t, stmts, 1) - stmt := stmts[0] - err = executor.ResetContextOfStmt(ctx, stmt) - require.NoError(t, err) - ret := &core.PreprocessorReturn{} - err = core.Preprocess(context.Background(), ctx, stmt, core.WithPreprocessorReturn(ret)) - require.NoError(t, err) - p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) - require.NoError(t, err) - planString := core.ToString(p) - testdata.OnRecord(func() { - output[i] = planString - }) - require.Equalf(t, output[i], planString, "case: %v", tt) - } -} - -func TestOutdatedAnalyze(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("create table t (a int, b int, index idx(a))") - for i := 0; i < 10; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d,%d)", i, i)) - } - h := dom.StatsHandle() - require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t") - testKit.MustExec("insert into t select * from t") - testKit.MustExec("insert into t select * from t") - testKit.MustExec("insert into t select * from t") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.Update(dom.InfoSchema())) - var input []struct { - SQL string - EnablePseudoForOutdatedStats bool - RatioOfPseudoEstimate float64 - } - var output []struct { - SQL string - EnablePseudoForOutdatedStats bool - RatioOfPseudoEstimate float64 - Plan []string - } - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testKit.Session().GetSessionVars().SetEnablePseudoForOutdatedStats(tt.EnablePseudoForOutdatedStats) - statistics.RatioOfPseudoEstimate.Store(tt.RatioOfPseudoEstimate) - plan := testKit.MustQuery(tt.SQL) - testdata.OnRecord(func() { - output[i].SQL = tt.SQL - output[i].EnablePseudoForOutdatedStats = tt.EnablePseudoForOutdatedStats - output[i].RatioOfPseudoEstimate = tt.RatioOfPseudoEstimate - output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) - }) - plan.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestNullCount(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("drop table if exists t") - testKit.MustExec("create table t (a int, b int, index idx(a))") - testKit.MustExec("insert into t values (null, null), (null, null)") - testKit.MustExec("analyze table t") - var input []string - var output [][]string - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - for i := 0; i < 2; i++ { - testdata.OnRecord(func() { - output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) - }) - testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) - } - h := dom.StatsHandle() - h.Clear() - require.NoError(t, h.Update(dom.InfoSchema())) - for i := 2; i < 4; i++ { - testdata.OnRecord(func() { - output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) - }) - testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) - } -} - -func TestCorrelatedEstimation(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 sql_mode='STRICT_TRANS_TABLES'") // disable only full group by - tk.MustExec("create table t(a int, b int, c int, index idx(c,b,a))") - tk.MustExec("insert into t values(1,1,1), (2,2,2), (3,3,3), (4,4,4), (5,5,5), (6,6,6), (7,7,7), (8,8,8), (9,9,9),(10,10,10)") - tk.MustExec("analyze table t") - var ( - input []string - output [][]string - ) - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - rs := tk.MustQuery(tt) - testdata.OnRecord(func() { - output[i] = testdata.ConvertRowsToStrings(rs.Rows()) - }) - rs.Check(testkit.Rows(output[i]...)) - } -} - -func TestInconsistentEstimation(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t(a int, b int, c int, index ab(a,b), index ac(a,c))") - tk.MustExec("insert into t values (1,1,1), (1000,1000,1000)") - for i := 0; i < 10; i++ { - tk.MustExec("insert into t values (5,5,5), (10,10,10)") - } - tk.MustExec("set @@tidb_analyze_version=1") - tk.MustExec("analyze table t with 2 buckets") - // Force using the histogram to estimate. - tk.MustExec("update mysql.stats_histograms set stats_ver = 0") - dom.StatsHandle().Clear() - require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema())) - var input []string - var output []struct { - SQL string - Plan []string - } - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - for i, sql := range input { - plan := tk.MustQuery(sql) - testdata.OnRecord(func() { - output[i].SQL = sql - output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) - }) - plan.Check(testkit.Rows(output[i].Plan...)) - } -} - func BenchmarkOptimize(b *testing.B) { store := testkit.CreateMockStore(b) @@ -632,37 +208,6 @@ func BenchmarkOptimize(b *testing.B) { } } -func TestIssue9562(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - var input [][]string - var output []struct { - SQL []string - Plan []string - } - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - for i, ts := range input { - for j, tt := range ts { - if j != len(ts)-1 { - tk.MustExec(tt) - } - testdata.OnRecord(func() { - output[i].SQL = ts - if j == len(ts)-1 { - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - } - }) - if j == len(ts)-1 { - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } - } - } -} - func TestIssue9805(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -691,70 +236,6 @@ func TestIssue9805(t *testing.T) { tk.MustQuery("explain analyze select /*+ TIDB_INLJ(t2) */ t1.id, t2.a from t1 join t2 on t1.a = t2.d where t1.b = 't2' and t1.d = 4") } -func TestLimitCrossEstimation(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("set @@session.tidb_executor_concurrency = 4;") - tk.MustExec("set @@session.tidb_hash_join_concurrency = 5;") - tk.MustExec("set @@session.tidb_distsql_scan_concurrency = 15;") - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int primary key, b int not null, c int not null default 0, index idx_bc(b, c))") - var input [][]string - var output []struct { - SQL []string - Plan []string - } - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - for i, ts := range input { - for j, tt := range ts { - if j != len(ts)-1 { - tk.MustExec(tt) - } - testdata.OnRecord(func() { - output[i].SQL = ts - if j == len(ts)-1 { - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - } - }) - if j == len(ts)-1 { - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } - } - } -} - -func TestLowSelIndexGreedySearch(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("set tidb_cost_model_version=2") - testKit.MustExec(`set tidb_opt_limit_push_down_threshold=0`) - testKit.MustExec("drop table if exists t") - testKit.MustExec("create table t (a varchar(32) default null, b varchar(10) default null, c varchar(12) default null, d varchar(32) default null, e bigint(10) default null, key idx1 (d,a), key idx2 (a,c), key idx3 (c,b), key idx4 (e))") - require.NoError(t, loadTableStats("analyzeSuiteTestLowSelIndexGreedySearchT.json", dom)) - var input []string - var output []struct { - SQL string - Plan []string - } - // The test purposes are: - // - index `idx2` runs much faster than `idx4` experimentally; - // - estimated row count of IndexLookUp should be 0; - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) - }) - testKit.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestUpdateProjEliminate(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -769,92 +250,6 @@ func TestUpdateProjEliminate(t *testing.T) { tk.MustExec("update tb1 set tb1.b=(select tb2.b from tb2 where tb2.a=tb1.a order by c desc limit 1);") } -func TestTiFlashCostModel(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("create table t (a int, b int, c int, primary key(a))") - tk.MustExec("insert into t values(1,1,1), (2,2,2), (3,3,3)") - - tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) - require.NoError(t, err) - // Set the hacked TiFlash replica for explain tests. - tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} - - var input, output [][]string - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - for i, ts := range input { - for j, tt := range ts { - if j != len(ts)-1 { - tk.MustExec(tt) - } - testdata.OnRecord(func() { - if j == len(ts)-1 { - output[i] = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - } - }) - if j == len(ts)-1 { - tk.MustQuery(tt).Check(testkit.Rows(output[i]...)) - } - } - } -} - -func TestIndexEqualUnknown(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("drop table if exists t, t1") - testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly - testKit.MustExec("CREATE TABLE t(a bigint(20) NOT NULL, b bigint(20) NOT NULL, c bigint(20) NOT NULL, PRIMARY KEY (a,c,b), KEY (b))") - require.NoError(t, loadTableStats("analyzeSuiteTestIndexEqualUnknownT.json", dom)) - var input []string - var output []struct { - SQL string - Plan []string - } - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) - }) - testKit.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestLimitIndexEstimation(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("drop table if exists t") - tk.MustExec("create table t(a int, b int, key idx_a(a), key idx_b(b))") - tk.MustExec("set session tidb_enable_extended_stats = on") - // Values in column a are from 1 to 1000000, values in column b are from 1000000 to 1, - // these 2 columns are strictly correlated in reverse order. - require.NoError(t, loadTableStats("analyzeSuiteTestLimitIndexEstimationT.json", dom)) - var input []string - var output []struct { - SQL string - Plan []string - } - - analyzeSuiteData := core.GetAnalyzeSuiteData() - analyzeSuiteData.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 TestBatchPointGetTablePartition(t *testing.T) { failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") From c5a44054740ec67eb40288a68474e31afd383136 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 15 Feb 2023 00:59:01 +0800 Subject: [PATCH 06/12] *: improve planner/core to make test faster Signed-off-by: Weizhen Wang --- planner/core/casetest/BUILD.bazel | 3 + planner/core/casetest/enforce_mpp_test.go | 489 ++++++++++++++++++++++ planner/core/enforce_mpp_test.go | 466 --------------------- 3 files changed, 492 insertions(+), 466 deletions(-) create mode 100644 planner/core/casetest/enforce_mpp_test.go diff --git a/planner/core/casetest/BUILD.bazel b/planner/core/casetest/BUILD.bazel index 7604ca5be0c88..d90ee661c89ca 100644 --- a/planner/core/casetest/BUILD.bazel +++ b/planner/core/casetest/BUILD.bazel @@ -6,6 +6,7 @@ go_test( srcs = [ "binary_plan_test.go", "cbo_test.go", + "enforce_mpp_test.go", "expression_rewriter_test.go", "main_test.go", "plan_test.go", @@ -37,6 +38,7 @@ go_test( "//planner/core/internal", "//planner/property", "//session", + "//sessionctx/stmtctx", "//sessionctx/variable", "//sessiontxn", "//statistics", @@ -46,6 +48,7 @@ go_test( "//testkit/testmain", "//testkit/testsetup", "//types", + "//util/collate", "//util/hint", "//util/logutil", "//util/mock", diff --git a/planner/core/casetest/enforce_mpp_test.go b/planner/core/casetest/enforce_mpp_test.go new file mode 100644 index 0000000000000..aaa2037efc0ce --- /dev/null +++ b/planner/core/casetest/enforce_mpp_test.go @@ -0,0 +1,489 @@ +// 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 ( + "strings" + "testing" + + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/util/collate" + "github.com/stretchr/testify/require" +) + +func TestEnforceMPP(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test query + 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)") + tk.MustExec("create index idx on t(a)") + + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + filterWarnings := func(originalWarnings []stmtctx.SQLWarn) []stmtctx.SQLWarn { + warnings := make([]stmtctx.SQLWarn, 0, 4) + for _, warning := range originalWarnings { + // filter out warning about skyline pruning + if !strings.Contains(warning.Err.Error(), "remain after pruning paths for") { + warnings = append(warnings, warning) + } + } + return warnings + } + 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].Warn = testdata.ConvertSQLWarnToStrings(filterWarnings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(filterWarnings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()))) + } +} + +// general cases. +func TestEnforceMPPWarning1(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test query + 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 as (a+1), c enum('xx', 'yy'), d bit(1))") + tk.MustExec("create index idx on t(a)") + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.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 + } + if strings.HasPrefix(tt, "cmd: create-replica") { + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: false, + } + } + } + continue + } + if strings.HasPrefix(tt, "cmd: enable-replica") { + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +// partition table. +func TestEnforceMPPWarning2(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test query + 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 char(20)) PARTITION BY HASH(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 { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.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].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +// new collation. +func TestEnforceMPPWarning3(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test query + 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 char(20))") + + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + if strings.HasPrefix(tt, "cmd: enable-new-collation") { + collate.SetNewCollationEnabledForTest(true) + continue + } + if strings.HasPrefix(tt, "cmd: disable-new-collation") { + collate.SetNewCollationEnabledForTest(false) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } + collate.SetNewCollationEnabledForTest(true) +} + +// Test enforce mpp warning for joins +func TestEnforceMPPWarning4(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test table + 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 primary key)") + tk.MustExec("drop table if exists s") + tk.MustExec("CREATE TABLE s(a int primary key)") + + // 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 == "t" || tblInfo.Name.L == "s" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +// Test agg push down for MPP mode +func TestMPP2PhaseAggPushDown(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test table + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists c") + tk.MustExec("drop table if exists o") + tk.MustExec("create table c(c_id bigint)") + tk.MustExec("create table o(o_id bigint, c_id bigint 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 == "c" || tblInfo.Name.L == "o" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +// Test skewed group distinct aggregate rewrite for MPP mode +func TestMPPSkewedGroupDistinctRewrite(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test table + 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 bigint not null, c bigint, d date, e varchar(20))") + + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +// Test 3 stage aggregation for single count distinct +func TestMPPSingleDistinct3Stage(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test table + 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 bigint not null, c bigint, d date, e varchar(20) collate utf8mb4_general_ci)") + + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} diff --git a/planner/core/enforce_mpp_test.go b/planner/core/enforce_mpp_test.go index dfd0a4fadbe3a..38ea191c432c8 100644 --- a/planner/core/enforce_mpp_test.go +++ b/planner/core/enforce_mpp_test.go @@ -17,16 +17,11 @@ package core_test import ( "fmt" "strconv" - "strings" "testing" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/parser/model" - plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/testkit/testdata" - "github.com/pingcap/tidb/util/collate" "github.com/stretchr/testify/require" ) @@ -84,464 +79,3 @@ func TestRowSizeInMPP(t *testing.T) { } require.True(t, costs[0] < costs[1] && costs[1] < costs[2]) // rowSize can affect the final cost } - -func TestEnforceMPP(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - // test query - 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)") - tk.MustExec("create index idx on t(a)") - - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - var input []string - var output []struct { - SQL string - Plan []string - Warn []string - } - enforceMPPSuiteData := plannercore.GetEnforceMPPSuiteData() - enforceMPPSuiteData.LoadTestCases(t, &input, &output) - filterWarnings := func(originalWarnings []stmtctx.SQLWarn) []stmtctx.SQLWarn { - warnings := make([]stmtctx.SQLWarn, 0, 4) - for _, warning := range originalWarnings { - // filter out warning about skyline pruning - if !strings.Contains(warning.Err.Error(), "remain after pruning paths for") { - warnings = append(warnings, warning) - } - } - return warnings - } - 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].Warn = testdata.ConvertSQLWarnToStrings(filterWarnings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(filterWarnings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()))) - } -} - -// general cases. -func TestEnforceMPPWarning1(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - // test query - 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 as (a+1), c enum('xx', 'yy'), d bit(1))") - tk.MustExec("create index idx on t(a)") - - var input []string - var output []struct { - SQL string - Plan []string - Warn []string - } - enforceMPPSuiteData := plannercore.GetEnforceMPPSuiteData() - enforceMPPSuiteData.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 - } - if strings.HasPrefix(tt, "cmd: create-replica") { - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: false, - } - } - } - continue - } - if strings.HasPrefix(tt, "cmd: enable-replica") { - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - continue - } - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} - -// partition table. -func TestEnforceMPPWarning2(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - // test query - 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 char(20)) PARTITION BY HASH(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 { - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - var input []string - var output []struct { - SQL string - Plan []string - Warn []string - } - enforceMPPSuiteData := plannercore.GetEnforceMPPSuiteData() - enforceMPPSuiteData.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].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} - -// new collation. -func TestEnforceMPPWarning3(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - // test query - 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 char(20))") - - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - var input []string - var output []struct { - SQL string - Plan []string - Warn []string - } - enforceMPPSuiteData := plannercore.GetEnforceMPPSuiteData() - enforceMPPSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - }) - if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { - tk.MustExec(tt) - continue - } - if strings.HasPrefix(tt, "cmd: enable-new-collation") { - collate.SetNewCollationEnabledForTest(true) - continue - } - if strings.HasPrefix(tt, "cmd: disable-new-collation") { - collate.SetNewCollationEnabledForTest(false) - continue - } - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } - collate.SetNewCollationEnabledForTest(true) -} - -// Test enforce mpp warning for joins -func TestEnforceMPPWarning4(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - // test table - 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 primary key)") - tk.MustExec("drop table if exists s") - tk.MustExec("CREATE TABLE s(a int primary key)") - - // 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 == "t" || tblInfo.Name.L == "s" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - var input []string - var output []struct { - SQL string - Plan []string - Warn []string - } - enforceMPPSuiteData := plannercore.GetEnforceMPPSuiteData() - enforceMPPSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - }) - if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { - tk.MustExec(tt) - continue - } - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} - -// Test agg push down for MPP mode -func TestMPP2PhaseAggPushDown(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - // test table - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists c") - tk.MustExec("drop table if exists o") - tk.MustExec("create table c(c_id bigint)") - tk.MustExec("create table o(o_id bigint, c_id bigint 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 == "c" || tblInfo.Name.L == "o" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - var input []string - var output []struct { - SQL string - Plan []string - Warn []string - } - enforceMPPSuiteData := plannercore.GetEnforceMPPSuiteData() - enforceMPPSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - }) - if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { - tk.MustExec(tt) - continue - } - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} - -// Test skewed group distinct aggregate rewrite for MPP mode -func TestMPPSkewedGroupDistinctRewrite(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - // test table - 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 bigint not null, c bigint, d date, e varchar(20))") - - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - var input []string - var output []struct { - SQL string - Plan []string - Warn []string - } - enforceMPPSuiteData := plannercore.GetEnforceMPPSuiteData() - enforceMPPSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - }) - if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { - tk.MustExec(tt) - continue - } - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} - -// Test 3 stage aggregation for single count distinct -func TestMPPSingleDistinct3Stage(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - // test table - 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 bigint not null, c bigint, d date, e varchar(20) collate utf8mb4_general_ci)") - - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - var input []string - var output []struct { - SQL string - Plan []string - Warn []string - } - enforceMPPSuiteData := plannercore.GetEnforceMPPSuiteData() - enforceMPPSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - }) - if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { - tk.MustExec(tt) - continue - } - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} From 29f8b5afb39adc00b9dd69319aed214a87ff9537 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 15 Feb 2023 01:35:32 +0800 Subject: [PATCH 07/12] *: improve planner/core to make test faster Signed-off-by: Weizhen Wang --- planner/core/BUILD.bazel | 6 +- planner/core/casetest/BUILD.bazel | 7 + planner/core/{ => casetest}/flat_plan_test.go | 4 +- .../casetest/integration_partition_test.go | 221 ++ planner/core/casetest/integration_test.go | 1828 ++++++++++++++ planner/core/casetest/physical_plan_test.go | 2193 +++++++++++++++++ planner/core/integration_partition_test.go | 199 -- planner/core/integration_test.go | 2063 +--------------- planner/core/internal/BUILD.bazel | 4 + planner/core/internal/testkit.go | 25 + planner/core/physical_plan_test.go | 2190 +--------------- types/BUILD.bazel | 1 + 12 files changed, 4405 insertions(+), 4336 deletions(-) rename planner/core/{ => casetest}/flat_plan_test.go (97%) create mode 100644 planner/core/casetest/integration_partition_test.go create mode 100644 planner/core/casetest/integration_test.go create mode 100644 planner/core/casetest/physical_plan_test.go diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index 82883f76f2df6..cecfad177ecdd 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -170,7 +170,6 @@ go_test( "expression_rewriter_test.go", "expression_test.go", "find_best_task_test.go", - "flat_plan_test.go", "fragment_test.go", "indexmerge_intersection_test.go", "indexmerge_path_test.go", @@ -231,6 +230,7 @@ go_test( "//parser/mysql", "//parser/terror", "//planner", + "//planner/core/internal", "//planner/property", "//planner/util", "//session", @@ -240,8 +240,6 @@ go_test( "//sessiontxn", "//statistics", "//statistics/handle", - "//store/mockstore", - "//store/mockstore/unistore", "//table", "//testkit", "//testkit/ddlhelper", @@ -270,12 +268,10 @@ go_test( "@com_github_golang_snappy//:snappy", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", - "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pingcap_tipb//go-tipb", "@com_github_prometheus_client_golang//prometheus", "@com_github_prometheus_client_model//go", "@com_github_stretchr_testify//require", - "@com_github_tikv_client_go_v2//testutils", "@org_golang_x_exp//slices", "@org_uber_go_goleak//:goleak", ], diff --git a/planner/core/casetest/BUILD.bazel b/planner/core/casetest/BUILD.bazel index d90ee661c89ca..4e4d9538bb9de 100644 --- a/planner/core/casetest/BUILD.bazel +++ b/planner/core/casetest/BUILD.bazel @@ -8,7 +8,11 @@ go_test( "cbo_test.go", "enforce_mpp_test.go", "expression_rewriter_test.go", + "flat_plan_test.go", + "integration_partition_test.go", + "integration_test.go", "main_test.go", + "physical_plan_test.go", "plan_test.go", "point_get_plan_test.go", "predicate_simplification_test.go", @@ -29,10 +33,12 @@ go_test( "//expression", "//expression/aggregation", "//infoschema", + "//kv", "//parser", "//parser/ast", "//parser/model", "//parser/mysql", + "//parser/terror", "//planner", "//planner/core", "//planner/core/internal", @@ -44,6 +50,7 @@ go_test( "//statistics", "//statistics/handle", "//testkit", + "//testkit/external", "//testkit/testdata", "//testkit/testmain", "//testkit/testsetup", diff --git a/planner/core/flat_plan_test.go b/planner/core/casetest/flat_plan_test.go similarity index 97% rename from planner/core/flat_plan_test.go rename to planner/core/casetest/flat_plan_test.go index 067bd8b63acd3..bfed5bed4c0a5 100644 --- a/planner/core/flat_plan_test.go +++ b/planner/core/casetest/flat_plan_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core_test +package casetest import ( "context" @@ -74,7 +74,7 @@ func TestFlatPhysicalPlan(t *testing.T) { Main []*FlatPhysicalOperatorForTest CTEs [][]*FlatPhysicalOperatorForTest } - planSuiteData := core.GetFlatPlanSuiteData() + planSuiteData := GetFlatPlanSuiteData() planSuiteData.LoadTestCases(t, &input, &output) p := parser.New() is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) diff --git a/planner/core/casetest/integration_partition_test.go b/planner/core/casetest/integration_partition_test.go new file mode 100644 index 0000000000000..182553717facd --- /dev/null +++ b/planner/core/casetest/integration_partition_test.go @@ -0,0 +1,221 @@ +// 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 ( + "strings" + "testing" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/stretchr/testify/require" +) + +func TestListPartitionPushDown(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database list_push_down") + tk.MustExec("use list_push_down") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists tlist") + tk.MustExec(`set tidb_enable_list_partition = 1`) + tk.MustExec(`create table tlist (a int) partition by list (a) ( + partition p0 values in (0, 1, 2), + partition p1 values in (3, 4, 5))`) + tk.MustExec(`create table tcollist (a int) partition by list columns(a) ( + partition p0 values in (0, 1, 2), + partition p1 values in (3, 4, 5))`) + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationPartitionSuiteData := GetIntegrationPartitionSuiteData() + integrationPartitionSuiteData.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 TestListColVariousTypes(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database list_col_partition_types") + tk.MustExec("use list_col_partition_types") + tk.MustExec("drop table if exists tlist") + tk.MustExec(`set tidb_enable_list_partition = 1`) + + tk.MustExec(`create table tint (a int) partition by list columns(a) (partition p0 values in (0, 1), partition p1 values in (2, 3))`) + tk.MustExec(`create table tdate (a date) partition by list columns(a) (partition p0 values in ('2000-01-01', '2000-01-02'), partition p1 values in ('2000-01-03', '2000-01-04'))`) + tk.MustExec(`create table tstring (a varchar(32)) partition by list columns(a) (partition p0 values in ('a', 'b'), partition p1 values in ('c', 'd'))`) + + err := tk.ExecToErr(`create table tdouble (a double) partition by list columns(a) (partition p0 values in (0, 1), partition p1 values in (2, 3))`) + require.Error(t, err) + require.Contains(t, err.Error(), "not allowed") + + err = tk.ExecToErr(`create table tdecimal (a decimal(30, 10)) partition by list columns(a) (partition p0 values in (0, 1), partition p1 values in (2, 3))`) + require.Error(t, err) + require.Contains(t, err.Error(), "not allowed") + + tk.MustExec(`insert into tint values (0), (1), (2), (3)`) + tk.MustExec(`insert into tdate values ('2000-01-01'), ('2000-01-02'), ('2000-01-03'), ('2000-01-04')`) + tk.MustExec(`insert into tstring values ('a'), ('b'), ('c'), ('d')`) + tk.MustExec(`analyze table tint`) + tk.MustExec(`analyze table tdate`) + tk.MustExec(`analyze table tstring`) + + var input []string + var output []struct { + SQL string + Results []string + } + integrationPartitionSuiteData := GetIntegrationPartitionSuiteData() + integrationPartitionSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Results = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Results...)) + } +} + +func TestListPartitionPruning(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("create database list_partition_pruning") + tk.MustExec("use list_partition_pruning") + tk.MustExec("drop table if exists tlist") + tk.MustExec(`set tidb_enable_list_partition = 1`) + tk.MustExec(`create table tlist (a int) partition by list (a) ( + partition p0 values in (0, 1, 2), + partition p1 values in (3, 4, 5), + partition p2 values in (6, 7, 8), + partition p3 values in (9, 10, 11))`) + tk.MustExec(`create table tcollist (a int) partition by list columns(a) ( + partition p0 values in (0, 1, 2), + partition p1 values in (3, 4, 5), + partition p2 values in (6, 7, 8), + partition p3 values in (9, 10, 11))`) + tk.MustExec(`analyze table tlist`) + tk.MustExec(`analyze table tcollist`) + + var input []string + var output []struct { + SQL string + DynamicPlan []string + StaticPlan []string + } + integrationPartitionSuiteData := GetIntegrationPartitionSuiteData() + integrationPartitionSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + output[i].DynamicPlan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") + output[i].StaticPlan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustQuery(tt).Check(testkit.Rows(output[i].DynamicPlan...)) + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") + tk.MustQuery(tt).Check(testkit.Rows(output[i].StaticPlan...)) + } +} + +func TestListPartitionFunctions(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database list_partition_pruning") + tk.MustExec("use list_partition_pruning") + tk.MustExec("set tidb_enable_list_partition = 1") + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") + + var input []string + var output []struct { + SQL string + Results []string + } + integrationPartitionSuiteData := GetIntegrationPartitionSuiteData() + integrationPartitionSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Results = nil + if strings.Contains(tt, "select") { + output[i].Results = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + } + }) + + if strings.Contains(tt, "select") { + tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Results...)) + } else { + tk.MustExec(tt) + } + } +} + +func TestEstimationForTopNPushToDynamicPartition(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 tlist") + tk.MustExec(`set tidb_enable_list_partition = 1`) + tk.MustExec(`create table trange (a int, b int, c int, index ia(a), primary key (b) clustered) + partition by range(b) ( + partition p1 values less than(100), + partition p2 values less than(200), + partition p3 values less than maxvalue);`) + tk.MustExec(`create table tlist (a int, b int, c int, index ia(a), primary key (b) clustered) + partition by list (b) ( + partition p0 values in (0, 1, 2), + partition p1 values in (3, 4, 5));`) + tk.MustExec(`create table thash (a int, b int, c int, index ia(a), primary key (b) clustered) + partition by hash(b) partitions 4;`) + tk.MustExec(`create table t (a int, b int, c int, index ia(a), primary key (b) clustered);`) + tk.MustExec(`analyze table trange;`) + tk.MustExec(`analyze table tlist;`) + tk.MustExec(`analyze table thash;`) + tk.MustExec(`analyze table t;`) + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationPartitionSuiteData := GetIntegrationPartitionSuiteData() + integrationPartitionSuiteData.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...)) + } +} diff --git a/planner/core/casetest/integration_test.go b/planner/core/casetest/integration_test.go new file mode 100644 index 0000000000000..07b0a2b9dd72e --- /dev/null +++ b/planner/core/casetest/integration_test.go @@ -0,0 +1,1828 @@ +// 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 ( + "strings" + "testing" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/planner/core/internal" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/stretchr/testify/require" +) + +func TestPushLimitDownIndexLookUpReader(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("set @@session.tidb_executor_concurrency = 4;") + tk.MustExec("set @@session.tidb_hash_join_concurrency = 5;") + tk.MustExec("set @@session.tidb_distsql_scan_concurrency = 15;") + tk.MustExec("use test") + tk.MustExec("drop table if exists tbl") + tk.MustExec("create table tbl(a int, b int, c int, key idx_b_c(b,c))") + tk.MustExec("insert into tbl values(1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5)") + tk.MustExec("analyze table tbl") + + 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 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 TestSelPushDownTiFlash(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 varchar(20))") + + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_allow_mpp = 0") + tk.MustExec("set tidb_cost_model_version=2") + + 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 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 TestPushDownToTiFlashWithKeepOrder(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 varchar(20))") + + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_allow_mpp = 0") + 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 TestPushDownToTiFlashWithKeepOrderInFastMode(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 varchar(20))") + tk.MustExec("set @@session.tiflash_fastscan=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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_allow_mpp = 0") + 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 TestMPPJoin(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 d1_t") + tk.MustExec("create table d1_t(d1_k int, value int)") + tk.MustExec("insert into d1_t values(1,2),(2,3)") + tk.MustExec("analyze table d1_t") + tk.MustExec("drop table if exists d2_t") + tk.MustExec("create table d2_t(d2_k decimal(10,2), value int)") + tk.MustExec("insert into d2_t values(10.11,2),(10.12,3)") + tk.MustExec("analyze table d2_t") + tk.MustExec("drop table if exists d3_t") + tk.MustExec("create table d3_t(d3_k date, value int)") + tk.MustExec("insert into d3_t values(date'2010-01-01',2),(date'2010-01-02',3)") + tk.MustExec("analyze table d3_t") + tk.MustExec("drop table if exists fact_t") + tk.MustExec("create table fact_t(d1_k int, d2_k decimal(10,2), d3_k date, col1 int, col2 int, col3 int)") + tk.MustExec("insert into fact_t values(1,10.11,date'2010-01-01',1,2,3),(1,10.11,date'2010-01-02',1,2,3),(1,10.12,date'2010-01-01',1,2,3),(1,10.12,date'2010-01-02',1,2,3)") + tk.MustExec("insert into fact_t values(2,10.11,date'2010-01-01',1,2,3),(2,10.11,date'2010-01-02',1,2,3),(2,10.12,date'2010-01-01',1,2,3),(2,10.12,date'2010-01-02',1,2,3)") + tk.MustExec("analyze table fact_t") + + // 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 == "fact_t" || tblInfo.Name.L == "d1_t" || tblInfo.Name.L == "d2_t" || tblInfo.Name.L == "d3_t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.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 TestMPPLeftSemiJoin(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + // test table + tk.MustExec("use test") + tk.MustExec("create table test.t(a int not null, b int null);") + tk.MustExec("set tidb_allow_mpp=1; set tidb_enforce_mpp=1;") + + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []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") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +func TestMPPOuterJoinBuildSideForBroadcastJoin(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 a") + tk.MustExec("create table a(id int, value int)") + tk.MustExec("insert into a values(1,2),(2,3)") + tk.MustExec("analyze table a") + tk.MustExec("drop table if exists b") + tk.MustExec("create table b(id int, value int)") + tk.MustExec("insert into b values(1,2),(2,3),(3,4)") + tk.MustExec("analyze table b") + // 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 == "a" || tblInfo.Name.L == "b" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 0") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 10000") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 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()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestMPPOuterJoinBuildSideForShuffleJoinWithFixedBuildSide(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 a") + tk.MustExec("create table a(id int, value int)") + tk.MustExec("insert into a values(1,2),(2,3)") + tk.MustExec("analyze table a") + tk.MustExec("drop table if exists b") + tk.MustExec("create table b(id int, value int)") + tk.MustExec("insert into b values(1,2),(2,3),(3,4)") + tk.MustExec("analyze table b") + // 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 == "a" || tblInfo.Name.L == "b" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 1") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 0") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 0") + 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 TestMPPOuterJoinBuildSideForShuffleJoin(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 a") + tk.MustExec("create table a(id int, value int)") + tk.MustExec("insert into a values(1,2),(2,3)") + tk.MustExec("analyze table a") + tk.MustExec("drop table if exists b") + tk.MustExec("create table b(id int, value int)") + tk.MustExec("insert into b values(1,2),(2,3),(3,4)") + tk.MustExec("analyze table b") + // 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 == "a" || tblInfo.Name.L == "b" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 0") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 0") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 0") + 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 TestMPPShuffledJoin(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 d1_t") + tk.MustExec("create table d1_t(d1_k int, value int)") + tk.MustExec("insert into d1_t values(1,2),(2,3)") + tk.MustExec("insert into d1_t values(1,2),(2,3)") + tk.MustExec("analyze table d1_t") + tk.MustExec("drop table if exists d2_t") + tk.MustExec("create table d2_t(d2_k decimal(10,2), value int)") + tk.MustExec("insert into d2_t values(10.11,2),(10.12,3)") + tk.MustExec("insert into d2_t values(10.11,2),(10.12,3)") + tk.MustExec("analyze table d2_t") + tk.MustExec("drop table if exists d3_t") + tk.MustExec("create table d3_t(d3_k date, value int)") + tk.MustExec("insert into d3_t values(date'2010-01-01',2),(date'2010-01-02',3)") + tk.MustExec("insert into d3_t values(date'2010-01-01',2),(date'2010-01-02',3)") + tk.MustExec("analyze table d3_t") + tk.MustExec("drop table if exists fact_t") + tk.MustExec("create table fact_t(d1_k int, d2_k decimal(10,2), d3_k date, col1 int, col2 int, col3 int)") + tk.MustExec("insert into fact_t values(1,10.11,date'2010-01-01',1,2,3),(1,10.11,date'2010-01-02',1,2,3),(1,10.12,date'2010-01-01',1,2,3),(1,10.12,date'2010-01-02',1,2,3)") + tk.MustExec("insert into fact_t values(2,10.11,date'2010-01-01',1,2,3),(2,10.11,date'2010-01-02',1,2,3),(2,10.12,date'2010-01-01',1,2,3),(2,10.12,date'2010-01-02',1,2,3)") + tk.MustExec("insert into fact_t values(2,10.11,date'2010-01-01',1,2,3),(2,10.11,date'2010-01-02',1,2,3),(2,10.12,date'2010-01-01',1,2,3),(2,10.12,date'2010-01-02',1,2,3)") + tk.MustExec("insert into fact_t values(2,10.11,date'2010-01-01',1,2,3),(2,10.11,date'2010-01-02',1,2,3),(2,10.12,date'2010-01-01',1,2,3),(2,10.12,date'2010-01-02',1,2,3)") + tk.MustExec("analyze table fact_t") + + // 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 == "fact_t" || tblInfo.Name.L == "d1_t" || tblInfo.Name.L == "d2_t" || tblInfo.Name.L == "d3_t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_allow_mpp = 1") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 1") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 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 TestMPPJoinWithCanNotFoundColumnInSchemaColumnsError(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") + tk.MustExec("create table t1(id int, v1 decimal(20,2), v2 decimal(20,2))") + tk.MustExec("create table t2(id int, v1 decimal(10,2), v2 decimal(10,2))") + tk.MustExec("create table t3(id int, v1 decimal(10,2), v2 decimal(10,2))") + tk.MustExec("insert into t1 values(1,1,1),(2,2,2)") + tk.MustExec("insert into t2 values(1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8)") + tk.MustExec("insert into t3 values(1,1,1)") + tk.MustExec("analyze table t1") + tk.MustExec("analyze table t2") + tk.MustExec("analyze table t3") + + 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.Name.L == "t3" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_enforce_mpp = 1") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 0") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 0") + tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 0") + + 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 TestJoinNotSupportedByTiFlash(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists table_1") + tk.MustExec("create table table_1(id int not null, bit_col bit(2) not null, datetime_col datetime not null)") + tk.MustExec("insert into table_1 values(1,b'1','2020-01-01 00:00:00'),(2,b'0','2020-01-01 00:00:00')") + tk.MustExec("analyze table table_1") + + tk.MustExec("insert into mysql.expr_pushdown_blacklist values('dayofmonth', 'tiflash', '');") + tk.MustExec("admin reload expr_pushdown_blacklist;") + + // 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 == "table_1" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.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...)) + } + + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 1") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 1") + 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 TestMPPWithHashExchangeUnderNewCollation(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists table_1") + tk.MustExec("create table table_1(id int not null, value char(10)) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci;") + tk.MustExec("insert into table_1 values(1,'1'),(2,'2')") + tk.MustExec("drop table if exists table_2") + tk.MustExec("create table table_2(id int not null, value char(10)) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin;") + tk.MustExec("insert into table_2 values(1,'1'),(2,'2')") + tk.MustExec("analyze table table_1") + tk.MustExec("analyze table table_2") + + // 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 == "table_1" || tblInfo.Name.L == "table_2" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_allow_mpp = 1") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 0") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 0") + tk.MustExec("set @@session.tidb_hash_exchange_with_new_collation = 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 TestMPPWithBroadcastExchangeUnderNewCollation(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists table_1") + tk.MustExec("create table table_1(id int not null, value char(10))") + tk.MustExec("insert into table_1 values(1,'1'),(2,'2')") + tk.MustExec("analyze table table_1") + + // 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 == "table_1" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.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 TestMPPAvgRewrite(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists table_1") + tk.MustExec("create table table_1(id int not null, value decimal(10,2))") + tk.MustExec("insert into table_1 values(1,1),(2,2)") + tk.MustExec("analyze table table_1") + + // 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 == "table_1" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.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 TestReadFromStorageHint(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, tt, ttt") + tk.MustExec("set session tidb_allow_mpp=OFF") + tk.MustExec("create table t(a int, b int, index ia(a))") + tk.MustExec("create table tt(a int, b int, primary key(a))") + tk.MustExec("create table ttt(a int, primary key (a desc))") + + // 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, + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []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].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +func TestKeepOrderHint(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, t1, th") + tk.MustExec("drop view if exists v, v1") + tk.MustExec("create table t(a int, b int, primary key(a));") + tk.MustExec("create table t1(a int, b int, index idx_a(a));") + tk.MustExec("create table th (a int, key(a)) partition by hash(a) partitions 4;") + tk.MustExec("create definer='root'@'localhost' view v as select * from t1 where a<10 order by a limit 1;") + tk.MustExec("create definer='root'@'localhost' view v1 as select * from t where a<10 order by a limit 1;") + + // If the optimizer can not generate the keep order plan, it will report error + err := tk.ExecToErr("explain select /*+ order_index(t1, idx_a) */ * from t1 where a<10 limit 1;") + require.EqualError(t, err, "[planner:1815]Internal : Can't find a proper physical plan for this query") + + err = tk.ExecToErr("explain select /*+ order_index(t, primary) */ * from t where a<10 limit 1;") + require.EqualError(t, err, "[planner:1815]Internal : Can't find a proper physical plan for this query") + + // The partition table can not keep order + tk.MustExec("analyze table th;") + err = tk.ExecToErr("select a from th where a<1 order by a limit 1;") + require.NoError(t, err) + + err = tk.ExecToErr("select /*+ order_index(th, a) */ a from th where a<1 order by a limit 1;") + require.EqualError(t, err, "[planner:1815]Internal : Can't find a proper physical plan for this query") + + var input []string + var output []struct { + SQL string + Plan []string + Warn []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].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +func TestViewHint(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 view if exists v, v1, v2") + tk.MustExec("drop table if exists t, t1, t2") + tk.MustExec("create table t(a int, b int);") + tk.MustExec("create table t1(a int, b int);") + tk.MustExec("create table t2(a int, b int);") + tk.MustExec("create definer='root'@'localhost' view v as select t.a, t.b from t join (select count(*) as a from t1 join t2 on t1.b=t2.b group by t2.a) tt on t.a = tt.a;") + tk.MustExec("create definer='root'@'localhost' view v1 as select t.a, t.b from t join (select count(*) as a from t1 join v on t1.b=v.b group by v.a) tt on t.a = tt.a;") + tk.MustExec("create definer='root'@'localhost' view v2 as select t.a, t.b from t join (select count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") + + var input []string + var output []struct { + SQL string + Plan []string + Warn []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].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +func TestViewHintScope(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 view if exists v, v1, v2, v3, v4") + tk.MustExec("drop table if exists t, t1, t2, t3, t4") + tk.MustExec("create table t(a int, b int);") + 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)") + tk.MustExec("create table t4(a int, b int, index idx_a(a), index idx_b(b))") + tk.MustExec("create definer='root'@'localhost' view v as select t.a, t.b from t join (select count(*) as a from t1 join t2 join t3 where t1.b=t2.b and t2.a = t3.a group by t2.a) tt on t.a = tt.a;") + tk.MustExec("create definer='root'@'localhost' view v1 as select t.a, t.b from t join (select count(*) as a from t1 join v on t1.b=v.b group by v.a) tt on t.a = tt.a;") + tk.MustExec("create definer='root'@'localhost' view v2 as select t.a, t.b from t join (select count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") + tk.MustExec("create definer='root'@'localhost' view v3 as select /*+ merge_join(t) */ t.a, t.b from t join (select /*+ stream_agg() */ count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") + tk.MustExec("create definer='root'@'localhost' view v4 as select * from t4 where a > 2 and b > 3;") + + var input []string + var output []struct { + SQL string + Plan []string + Warn []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].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +func TestAllViewHintType(t *testing.T) { + store := testkit.CreateMockStore(t, internal.WithMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("set @@session.tidb_allow_mpp=ON") + tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash, tikv'") + tk.MustExec("drop view if exists v, v1, v2, v3, v4, v5, v6, v7, v8, v9, v10, v11, v12") + tk.MustExec("drop table if exists t, t1, t2, t4, t3, t5") + tk.MustExec("create table t(a int not null, b int, index idx_a(a));") + tk.MustExec("create table t1(a int not null, b int, index idx_a(a));") + tk.MustExec("create table t2(a int, b int, index idx_a(a));") + tk.MustExec("create table t3(a int, b int, index idx_a(a));") + tk.MustExec("create table t4(a int, b int, index idx_a(a));") + tk.MustExec("create table t5(a int, b int, index idx_a(a), index idx_b(b));") + + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("create definer='root'@'localhost' view v as select t.a, t.b from t join t1 on t.a = t1.a;") + tk.MustExec("create definer='root'@'localhost' view v1 as select t2.a, t2.b from t2 join t3 join v where t2.b = t3.b and t3.a = v.a;") + tk.MustExec("create definer='root'@'localhost' view v2 as select t.a, t.b from t join (select count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") + tk.MustExec("create definer='root'@'localhost' view v3 as select * from t5 where a > 1 and b < 2;") + tk.MustExec("create definer='root'@'localhost' view v4 as select * from t5 where a > 1 or b < 2;") + tk.MustExec("create definer='root'@'localhost' view v5 as SELECT * FROM t WHERE EXISTS (SELECT 1 FROM t1 WHERE t1.b = t.b);") + tk.MustExec("create definer='root'@'localhost' view v6 as select * from t1 where t1.a < (select sum(t2.a) from t2 where t2.b = t1.b);") + tk.MustExec("create definer='root'@'localhost' view v7 as WITH CTE AS (SELECT * FROM t WHERE t.a < 60) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;") + tk.MustExec("create definer='root'@'localhost' view v8 as WITH CTE1 AS (SELECT b FROM t1), CTE2 AS (WITH CTE3 AS (SELECT a FROM t2), CTE4 AS (SELECT a FROM t3) SELECT CTE3.a FROM CTE3, CTE4) SELECT b FROM CTE1, CTE2 union select * from CTE1;") + tk.MustExec("create definer='root'@'localhost' view v9 as select sum(a) from t;") + tk.MustExec("create definer='root'@'localhost' view v10 as SELECT * FROM t WHERE a > 10 ORDER BY b LIMIT 1;") + tk.MustExec("create definer='root'@'localhost' view v11 as select a, sum(b) from t group by a") + tk.MustExec("create definer='root'@'localhost' view v12 as select t.a, t.b from t join t t1 on t.a = t1.a;") + + var input []string + var output []struct { + SQL string + Plan []string + Warn []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].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +func TestReadFromStorageHintAndIsolationRead(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, tt, ttt") + tk.MustExec("create table t(a int, b int, index ia(a))") + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tikv\"") + + // 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, + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +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 TestIsolationReadTiFlashUseIndexHint(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, index idx(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 + Plan []string + Warn []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].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +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...)) + } +} + +func TestIndexMergeHint4CNF(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(id int primary key, a int, b int, c int, key(a), key(b), 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 TestIndexHintWarning(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, key a(a), key b(b))") + tk.MustExec("create table t2(a int, b int, c int, key a(a), key b(b))") + var input []string + var output []struct { + SQL string + Warnings []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + tk.MustQuery(tt) + warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + output[i].Warnings = make([]string, len(warns)) + for j := range warns { + output[i].Warnings[j] = warns[j].Err.Error() + } + }) + tk.MustQuery(tt) + warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Len(t, warns, len(output[i].Warnings)) + for j := range warns { + require.Equal(t, stmtctx.WarnLevelWarning, warns[j].Level) + require.EqualError(t, warns[j].Err, output[i].Warnings[j]) + } + } + //Test view with index hint should result error + tk.MustExec("drop table if exists t1") + tk.MustExec("drop view if exists v1") + tk.MustExec("CREATE TABLE t1 (c1 INT PRIMARY KEY, c2 INT, INDEX (c2))") + tk.MustExec("INSERT INTO t1 VALUES (1,1), (2,2), (3,3)") + tk.MustExec("CREATE VIEW v1 AS SELECT c1, c2 FROM t1") + err := tk.ExecToErr("SELECT * FROM v1 USE INDEX (PRIMARY) WHERE c1=2") + require.True(t, terror.ErrorEqual(err, core.ErrKeyDoesNotExist)) +} + +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 TestHintWithRequiredProperty(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("set @@session.tidb_executor_concurrency = 4;") + tk.MustExec("set @@session.tidb_hash_join_concurrency = 5;") + tk.MustExec("set @@session.tidb_distsql_scan_concurrency = 15;") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int primary key, b int, c int, key b(b))") + 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()) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + output[i].Warnings = make([]string, len(warnings)) + for j, warning := range warnings { + output[i].Warnings[j] = warning.Err.Error() + } + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Len(t, warnings, len(output[i].Warnings)) + for j, warning := range warnings { + require.EqualError(t, warning.Err, output[i].Warnings[j]) + } + } +} + +func TestHintWithoutTableWarning(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, key a(a))") + tk.MustExec("create table t2(a int, b int, c int, key a(a))") + var input []string + var output []struct { + SQL string + Warnings []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + tk.MustQuery(tt) + warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + output[i].Warnings = make([]string, len(warns)) + for j := range warns { + output[i].Warnings[j] = warns[j].Err.Error() + } + }) + tk.MustQuery(tt) + warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Len(t, warns, len(output[i].Warnings)) + for j := range warns { + require.Equal(t, stmtctx.WarnLevelWarning, warns[j].Level) + require.EqualError(t, warns[j].Err, output[i].Warnings[j]) + } + } +} + +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 TestOptimizeHintOnPartitionTable(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 varchar(20), + primary key(a), key(b), key(c) + ) partition by range columns(a) ( + partition p0 values less than(6), + partition p1 values less than(11), + partition p2 values less than(16));`) + tk.MustExec(`insert into t values (1,1,"1"), (2,2,"2"), (8,8,"8"), (11,11,"11"), (15,15,"15")`) + tk.MustExec("set @@tidb_enable_index_merge = off") + defer func() { + tk.MustExec("set @@tidb_enable_index_merge = 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) + + var input []string + var output []struct { + SQL string + Plan []string + Warn []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].Warn = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warn...)) + } +} + +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...)) + } +} diff --git a/planner/core/casetest/physical_plan_test.go b/planner/core/casetest/physical_plan_test.go new file mode 100644 index 0000000000000..f3c66721227e3 --- /dev/null +++ b/planner/core/casetest/physical_plan_test.go @@ -0,0 +1,2193 @@ +// 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 ( + "context" + "fmt" + "strings" + "testing" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/planner" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/planner/core/internal" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/sessiontxn" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/external" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/util/hint" + "github.com/stretchr/testify/require" +) + +func assertSameHints(t *testing.T, expected, actual []*ast.TableOptimizerHint) { + expectedStr := make([]string, 0, len(expected)) + actualStr := make([]string, 0, len(actual)) + for _, h := range expected { + expectedStr = append(expectedStr, hint.RestoreTableOptimizerHint(h)) + } + for _, h := range actual { + actualStr = append(actualStr, hint.RestoreTableOptimizerHint(h)) + } + require.ElementsMatch(t, expectedStr, actualStr) +} + +func doTestPushdownDistinct(t *testing.T, vars, input []string, output []struct { + SQL string + Plan []string + Result []string +}) { + 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, index(c))") + tk.MustExec("insert into t values (1, 1, 1), (1, 1, 3), (1, 2, 3), (2, 1, 3), (1, 2, NULL);") + + tk.MustExec("drop table if exists pt") + tk.MustExec(`CREATE TABLE pt (a int, b int) PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (2), + PARTITION p1 VALUES LESS THAN (100) + );`) + + tk.MustExec("drop table if exists tc;") + tk.MustExec("CREATE TABLE `tc`(`timestamp` timestamp NULL DEFAULT NULL, KEY `idx_timestamp` (`timestamp`)) PARTITION BY RANGE ( UNIX_TIMESTAMP(`timestamp`) ) (PARTITION `p2020072312` VALUES LESS THAN (1595480400),PARTITION `p2020072313` VALUES LESS THAN (1595484000));") + + tk.MustExec("drop table if exists ta") + tk.MustExec("create table ta(a int);") + tk.MustExec("insert into ta values(1), (1);") + tk.MustExec("drop table if exists tb") + tk.MustExec("create table tb(a int);") + tk.MustExec("insert into tb values(1), (1);") + + tk.MustExec("set session sql_mode=''") + tk.MustExec(fmt.Sprintf("set session %s=1", variable.TiDBHashAggPartialConcurrency)) + tk.MustExec(fmt.Sprintf("set session %s=1", variable.TiDBHashAggFinalConcurrency)) + + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) + + for _, v := range vars { + tk.MustExec(v) + } + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) + } +} + +func TestDAGPlanBuilderSimpleCase(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") + var input []string + var output []struct { + SQL string + Best string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, tt := range input { + comment := fmt.Sprintf("case: %v, sql: %s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + require.NoError(t, sessiontxn.NewTxn(context.Background(), tk.Session())) + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Best = core.ToString(p) + }) + require.Equal(t, output[i].Best, core.ToString(p), comment) + } +} + +func TestDAGPlanBuilderJoin(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + sessionVars := tk.Session().GetSessionVars() + sessionVars.ExecutorConcurrency = 4 + sessionVars.SetDistSQLScanConcurrency(15) + sessionVars.SetHashJoinConcurrency(5) + + var input []string + var output []struct { + SQL string + Best string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, tt := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Best = core.ToString(p) + }) + require.Equal(t, output[i].Best, core.ToString(p), comment) + } +} + +func TestDAGPlanBuilderSubquery(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 sql_mode='STRICT_TRANS_TABLES'") // disable only full group by + sessionVars := tk.Session().GetSessionVars() + sessionVars.SetHashAggFinalConcurrency(1) + sessionVars.SetHashAggPartialConcurrency(1) + sessionVars.SetHashJoinConcurrency(5) + sessionVars.SetDistSQLScanConcurrency(15) + sessionVars.ExecutorConcurrency = 4 + var input []string + var output []struct { + SQL string + Best string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, tt := range input { + comment := fmt.Sprintf("input: %s", tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Best = core.ToString(p) + }) + require.Equal(t, output[i].Best, core.ToString(p), fmt.Sprintf("input: %s", tt)) + } +} + +func TestDAGPlanTopN(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + var input []string + var output []struct { + SQL string + Best string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, tt := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Best = core.ToString(p) + }) + require.Equal(t, output[i].Best, core.ToString(p), comment) + } +} + +func TestDAGPlanBuilderBasePhysicalPlan(t *testing.T) { + store := testkit.CreateMockStore(t) + + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + _, err = se.Execute(context.Background(), "use test") + require.NoError(t, err) + + var input []string + var output []struct { + SQL string + Best string + Hints string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, tt := range input { + comment := fmt.Sprintf("input: %s", tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + err = core.Preprocess(context.Background(), se, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: is})) + require.NoError(t, err) + p, _, err := planner.Optimize(context.TODO(), se, stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Best = core.ToString(p) + output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) + }) + require.Equal(t, output[i].Best, core.ToString(p), fmt.Sprintf("input: %s", tt)) + hints := core.GenHintsFromPhysicalPlan(p) + + // test the new genHints code + flat := core.FlattenPhysicalPlan(p, false) + newHints := core.GenHintsFromFlatPlan(flat) + assertSameHints(t, hints, newHints) + + require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(hints), fmt.Sprintf("input: %s", tt)) + } +} + +func TestDAGPlanBuilderUnion(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + var input []string + var output []struct { + SQL string + Best string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, tt := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Best = core.ToString(p) + }) + require.Equal(t, output[i].Best, core.ToString(p), comment) + } +} + +func TestDAGPlanBuilderUnionScan(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)") + + var input []string + var output []struct { + SQL string + Best string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + p := parser.New() + for i, tt := range input { + tk.MustExec("begin;") + tk.MustExec("insert into t values(2, 2, 2);") + + comment := fmt.Sprintf("input: %s", tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + dom := domain.GetDomain(tk.Session()) + require.NoError(t, dom.Reload()) + plan, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, dom.InfoSchema()) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Best = core.ToString(plan) + }) + require.Equal(t, output[i].Best, core.ToString(plan), fmt.Sprintf("input: %s", tt)) + tk.MustExec("rollback;") + } +} + +func TestDAGPlanBuilderAgg(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 sql_mode='STRICT_TRANS_TABLES'") // disable only full group by + sessionVars := tk.Session().GetSessionVars() + sessionVars.SetHashAggFinalConcurrency(1) + sessionVars.SetHashAggPartialConcurrency(1) + sessionVars.SetDistSQLScanConcurrency(15) + sessionVars.ExecutorConcurrency = 4 + + var input []string + var output []struct { + SQL string + Best string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, tt := range input { + comment := fmt.Sprintf("input: %s", tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Best = core.ToString(p) + }) + require.Equal(t, output[i].Best, core.ToString(p), fmt.Sprintf("input: %s", tt)) + } +} + +func TestRefine(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + + var input []string + var output []struct { + SQL string + Best string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, tt := range input { + comment := fmt.Sprintf("input: %s", tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + sc := tk.Session().GetSessionVars().StmtCtx + sc.IgnoreTruncate = false + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err, comment) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Best = core.ToString(p) + }) + require.Equal(t, output[i].Best, core.ToString(p), comment) + } +} + +func TestAggEliminator(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("set sql_mode='STRICT_TRANS_TABLES'") // disable only full group by + var input []string + var output []struct { + SQL string + Best string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, tt := range input { + comment := fmt.Sprintf("input: %s", tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + sc := tk.Session().GetSessionVars().StmtCtx + sc.IgnoreTruncate = false + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Best = core.ToString(p) + }) + require.Equal(t, output[i].Best, core.ToString(p), fmt.Sprintf("input: %s", tt)) + } +} + +func TestINMJHint(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + 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 primary key, b int not null)") + tk.MustExec("create table t2(a int primary key, b int not null)") + tk.MustExec("insert into t1 values(1,1),(2,2)") + tk.MustExec("insert into t2 values(1,1),(2,1)") + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) + } +} + +func TestEliminateMaxOneRow(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1;") + tk.MustExec("drop table if exists t2;") + tk.MustExec("drop table if exists t3;") + tk.MustExec("create table t1(a int(11) DEFAULT NULL, b int(11) DEFAULT NULL, UNIQUE KEY idx_a (a))") + tk.MustExec("create table t2(a int(11) DEFAULT NULL, b int(11) DEFAULT NULL)") + tk.MustExec("create table t3(a int(11) DEFAULT NULL, b int(11) DEFAULT NULL, c int(11) DEFAULT NULL, UNIQUE KEY idx_abc (a, b, c))") + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Check(testkit.Rows(output[i].Result...)) + } +} + +func TestIndexJoinUnionScan(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("create table t (a int primary key, b int, index idx(a))") + tk.MustExec("create table tt (a int primary key) partition by range (a) (partition p0 values less than (100), partition p1 values less than (200))") + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) + + var input [][]string + var output []struct { + SQL []string + Plan []string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + for i, ts := range input { + tk.MustExec("begin") + for j, tt := range ts { + if j != len(ts)-1 { + tk.MustExec(tt) + } + testdata.OnRecord(func() { + output[i].SQL = ts + if j == len(ts)-1 { + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + } + }) + if j == len(ts)-1 { + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } + } + tk.MustExec("rollback") + } +} + +func TestMergeJoinUnionScan(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1 (c_int int, c_str varchar(40), primary key (c_int))") + tk.MustExec("create table t2 (c_int int, c_str varchar(40), primary key (c_int))") + tk.MustExec("insert into t1 (`c_int`, `c_str`) values (11, 'keen williamson'), (10, 'gracious hermann')") + tk.MustExec("insert into t2 (`c_int`, `c_str`) values (10, 'gracious hermann')") + + var input [][]string + var output []struct { + SQL []string + Plan []string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + for i, ts := range input { + tk.MustExec("begin") + for j, tt := range ts { + if j != len(ts)-1 { + tk.MustExec(tt) + } + testdata.OnRecord(func() { + output[i].SQL = ts + if j == len(ts)-1 { + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + } + }) + if j == len(ts)-1 { + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } + } + tk.MustExec("rollback") + } +} + +func TestSemiJoinToInner(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + + var input []string + var output []struct { + SQL string + Best string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, tt := range input { + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err) + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Best = core.ToString(p) + }) + require.Equal(t, output[i].Best, core.ToString(p)) + } +} + +func TestUnmatchedTableInHint(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + var input []string + var output []struct { + SQL string + Warning string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, test := range input { + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) + stmt, err := p.ParseOneStmt(test, "", "") + require.NoError(t, err) + _, _, err = planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + testdata.OnRecord(func() { + output[i].SQL = test + if len(warnings) > 0 { + output[i].Warning = warnings[0].Err.Error() + } + }) + if output[i].Warning == "" { + require.Len(t, warnings, 0) + } else { + require.Len(t, warnings, 1) + require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) + require.Equal(t, output[i].Warning, warnings[0].Err.Error()) + } + } +} + +func TestMPPHints(t *testing.T) { + store := testkit.CreateMockStore(t, internal.WithMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("create table t (a int, b int, c int, index idx_a(a), index idx_b(b))") + tk.MustExec("alter table t set tiflash replica 1") + tk.MustExec("set @@session.tidb_allow_mpp=ON") + tk.MustExec("create definer='root'@'localhost' view v as select a, sum(b) from t group by a, c;") + tk.MustExec("create definer='root'@'localhost' view v1 as select t1.a from t t1, t t2 where t1.a=t2.a;") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +func TestHintScope(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + + var input []string + var output []struct { + SQL string + Best string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + + for i, test := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, test) + stmt, err := p.ParseOneStmt(test, "", "") + require.NoError(t, err, comment) + + p, _, err := planner.Optimize(context.Background(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = test + output[i].Best = core.ToString(p) + }) + require.Equal(t, output[i].Best, core.ToString(p)) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Len(t, warnings, 0, comment) + } +} + +func TestJoinHints(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + + var input []string + var output []struct { + SQL string + Best string + Warning string + Hints string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + ctx := context.Background() + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + + for i, test := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, test) + stmt, err := p.ParseOneStmt(test, "", "") + require.NoError(t, err, comment) + + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + + testdata.OnRecord(func() { + output[i].SQL = test + output[i].Best = core.ToString(p) + if len(warnings) > 0 { + output[i].Warning = warnings[0].Err.Error() + } + output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) + }) + require.Equal(t, output[i].Best, core.ToString(p)) + if output[i].Warning == "" { + require.Len(t, warnings, 0) + } else { + require.Len(t, warnings, 1, fmt.Sprintf("%v", warnings)) + require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) + require.Equal(t, output[i].Warning, warnings[0].Err.Error()) + } + hints := core.GenHintsFromPhysicalPlan(p) + + // test the new genHints code + flat := core.FlattenPhysicalPlan(p, false) + newHints := core.GenHintsFromFlatPlan(flat) + assertSameHints(t, hints, newHints) + + require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(hints), comment) + } +} + +func TestAggregationHints(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + sessionVars := tk.Session().GetSessionVars() + sessionVars.SetHashAggFinalConcurrency(1) + sessionVars.SetHashAggPartialConcurrency(1) + + var input []struct { + SQL string + AggPushDown bool + } + var output []struct { + SQL string + Best string + Warning string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + ctx := context.Background() + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, test := range input { + comment := fmt.Sprintf("case: %v sql: %v", i, test) + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) + tk.Session().GetSessionVars().AllowAggPushDown = test.AggPushDown + + stmt, err := p.ParseOneStmt(test.SQL, "", "") + require.NoError(t, err, comment) + + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + + testdata.OnRecord(func() { + output[i].SQL = test.SQL + output[i].Best = core.ToString(p) + if len(warnings) > 0 { + output[i].Warning = warnings[0].Err.Error() + } + }) + require.Equal(t, output[i].Best, core.ToString(p), comment) + if output[i].Warning == "" { + require.Len(t, warnings, 0) + } else { + require.Len(t, warnings, 1, fmt.Sprintf("%v", warnings)) + require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) + require.Equal(t, output[i].Warning, warnings[0].Err.Error()) + } + } +} + +func TestSemiJoinRewriteHints(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("create table t(a int, b int, c int)") + + sessionVars := tk.Session().GetSessionVars() + sessionVars.SetHashAggFinalConcurrency(1) + sessionVars.SetHashAggPartialConcurrency(1) + + var input []string + var output []struct { + SQL string + Plan []string + Warning string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + ctx := context.Background() + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, test := range input { + comment := fmt.Sprintf("case: %v sql: %v", i, test) + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) + + stmt, err := p.ParseOneStmt(test, "", "") + require.NoError(t, err, comment) + + _, _, err = planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + + testdata.OnRecord(func() { + output[i].SQL = test + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief'" + test).Rows()) + if len(warnings) > 0 { + output[i].Warning = warnings[0].Err.Error() + } + }) + tk.MustQuery("explain format = 'brief'" + test).Check(testkit.Rows(output[i].Plan...)) + if output[i].Warning == "" { + require.Len(t, warnings, 0) + } else { + require.Len(t, warnings, 1, fmt.Sprintf("%v", warnings)) + require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) + require.Equal(t, output[i].Warning, warnings[0].Err.Error()) + } + } +} + +func TestAggToCopHint(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists ta") + tk.MustExec("create table ta(a int, b int, index(a))") + + var ( + input []string + output []struct { + SQL string + Best string + Warning string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + ctx := context.Background() + is := domain.GetDomain(tk.Session()).InfoSchema() + p := parser.New() + for i, test := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, test) + testdata.OnRecord(func() { + output[i].SQL = test + }) + require.Equal(t, output[i].SQL, test, comment) + + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) + + stmt, err := p.ParseOneStmt(test, "", "") + require.NoError(t, err, comment) + + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err, comment) + planString := core.ToString(p) + testdata.OnRecord(func() { + output[i].Best = planString + }) + require.Equal(t, output[i].Best, planString, comment) + + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + testdata.OnRecord(func() { + if len(warnings) > 0 { + output[i].Warning = warnings[0].Err.Error() + } + }) + if output[i].Warning == "" { + require.Len(t, warnings, 0) + } else { + require.Len(t, warnings, 1, fmt.Sprintf("%v", warnings)) + require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) + require.Equal(t, output[i].Warning, warnings[0].Err.Error()) + } + } +} + +func TestLimitToCopHint(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 tn") + tk.MustExec("create table tn(a int, b int, c int, d int, key (a, b, c, d))") + tk.MustExec(`set tidb_opt_limit_push_down_threshold=0`) + + var ( + input []string + output []struct { + SQL string + Plan []string + Warning []string + } + ) + + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + + comment := fmt.Sprintf("case:%v sql:%s", i, ts) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + testdata.OnRecord(func() { + if len(warnings) > 0 { + output[i].Warning = make([]string, len(warnings)) + for j, warning := range warnings { + output[i].Warning[j] = warning.Err.Error() + } + } + }) + if len(output[i].Warning) == 0 { + require.Len(t, warnings, 0) + } else { + require.Len(t, warnings, len(output[i].Warning), comment) + for j, warning := range warnings { + require.Equal(t, stmtctx.WarnLevelWarning, warning.Level, comment) + require.Equal(t, output[i].Warning[j], warning.Err.Error(), comment) + } + } + } +} + +func TestCTEMergeHint(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists tc") + tk.MustExec("drop table if exists te") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("drop table if exists t3") + tk.MustExec("drop table if exists t4") + tk.MustExec("drop view if exists v") + tk.MustExec("create table tc(a int)") + tk.MustExec("create table te(c int)") + tk.MustExec("create table t1(a int)") + tk.MustExec("create table t2(b int)") + tk.MustExec("create table t3(c int)") + tk.MustExec("create table t4(d int)") + tk.MustExec("insert into tc values (1), (5), (10), (15), (20), (30), (50);") + tk.MustExec("insert into te values (1), (5), (10), (25), (40), (60), (100);") + tk.MustExec("insert into t1 values (1), (5), (10), (25), (40), (60), (100);") + tk.MustExec("insert into t2 values (1), (5), (10), (25), (40), (60), (100);") + tk.MustExec("insert into t3 values (1), (5), (10), (25), (40), (60), (100);") + tk.MustExec("insert into t4 values (1), (5), (10), (25), (40), (60), (100);") + tk.MustExec("analyze table tc;") + tk.MustExec("analyze table te;") + tk.MustExec("analyze table t1;") + tk.MustExec("analyze table t2;") + tk.MustExec("analyze table t3;") + tk.MustExec("analyze table t4;") + tk.MustExec("create definer='root'@'localhost' view v as select * from tc") + var ( + input []string + output []struct { + SQL string + Plan []string + Warning []string + } + ) + + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + + comment := fmt.Sprintf("case:%v sql:%s", i, ts) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + testdata.OnRecord(func() { + if len(warnings) > 0 { + output[i].Warning = make([]string, len(warnings)) + for j, warning := range warnings { + output[i].Warning[j] = warning.Err.Error() + } + } + }) + if len(output[i].Warning) == 0 { + require.Len(t, warnings, 0) + } else { + require.Len(t, warnings, len(output[i].Warning), comment) + for j, warning := range warnings { + require.Equal(t, stmtctx.WarnLevelWarning, warning.Level, comment) + require.Equal(t, output[i].Warning[j], warning.Err.Error(), comment) + } + } + } +} + +func TestForceInlineCTE(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(11));") + tk.MustExec("insert into t values (1), (5), (10), (15), (20), (30), (50);") + + var ( + input []string + output []struct { + SQL string + Plan []string + Warning []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + }) + if strings.HasPrefix(ts, "set") { + tk.MustExec(ts) + continue + } + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + ts).Rows()) + }) + tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + + comment := fmt.Sprintf("case:%v sql:%s", i, ts) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + testdata.OnRecord(func() { + if len(warnings) > 0 { + output[i].Warning = make([]string, len(warnings)) + for j, warning := range warnings { + output[i].Warning[j] = warning.Err.Error() + } + } + }) + if len(output[i].Warning) == 0 { + require.Len(t, warnings, 0) + } else { + require.Len(t, warnings, len(output[i].Warning), comment) + for j, warning := range warnings { + require.Equal(t, stmtctx.WarnLevelWarning, warning.Level, comment) + require.Equal(t, output[i].Warning[j], warning.Err.Error(), comment) + } + } + } +} + +func TestSingleConsumerCTE(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(11));") + tk.MustExec("insert into t values (1), (5), (10), (15), (20), (30), (50);") + + var ( + input []string + output []struct { + SQL string + Plan []string + Warning []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + }) + if strings.HasPrefix(ts, "set") { + tk.MustExec(ts) + continue + } + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + ts).Rows()) + }) + tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestPushdownDistinctEnable(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + vars := []string{ + fmt.Sprintf("set @@session.%s = 1", variable.TiDBOptDistinctAggPushDown), + "set session tidb_opt_agg_push_down = 1", + "set tidb_cost_model_version = 2", + } + doTestPushdownDistinct(t, vars, input, output) +} + +func TestPushdownDistinctDisable(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + vars := []string{ + fmt.Sprintf("set @@session.%s = 0", variable.TiDBOptDistinctAggPushDown), + "set session tidb_opt_agg_push_down = 1", + } + doTestPushdownDistinct(t, vars, input, output) +} + +func TestPushdownDistinctEnableAggPushDownDisable(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + vars := []string{ + fmt.Sprintf("set @@session.%s = 1", variable.TiDBOptDistinctAggPushDown), + "set session tidb_opt_agg_push_down = 0", + "set tidb_cost_model_version=2", + } + doTestPushdownDistinct(t, vars, input, output) +} + +func TestGroupConcatOrderby(t *testing.T) { + failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) + defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + 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, name int)") + tk.MustExec("insert into test values(1, 10);") + tk.MustExec("insert into test values(1, 20);") + tk.MustExec("insert into test values(1, 30);") + tk.MustExec("insert into test values(2, 20);") + tk.MustExec("insert into test values(3, 200);") + tk.MustExec("insert into test values(3, 500);") + + tk.MustExec("drop table if exists ptest;") + tk.MustExec("CREATE TABLE ptest (id int,name int) PARTITION BY RANGE ( id ) " + + "(PARTITION `p0` VALUES LESS THAN (2), PARTITION `p1` VALUES LESS THAN (11))") + tk.MustExec("insert into ptest select * from test;") + tk.MustExec(fmt.Sprintf("set session tidb_opt_distinct_agg_push_down = %v", 1)) + tk.MustExec(fmt.Sprintf("set session tidb_opt_agg_push_down = %v", 1)) + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Check(testkit.Rows(output[i].Result...)) + } +} + +func TestIndexHint(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + var input []string + var output []struct { + SQL string + Best string + HasWarn bool + Hints string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + ctx := context.Background() + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + + for i, test := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, test) + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) + + stmt, err := p.ParseOneStmt(test, "", "") + require.NoError(t, err, comment) + + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = test + output[i].Best = core.ToString(p) + output[i].HasWarn = len(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) > 0 + output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) + }) + require.Equal(t, output[i].Best, core.ToString(p), comment) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + if output[i].HasWarn { + require.Len(t, warnings, 1, comment) + } else { + require.Len(t, warnings, 0, comment) + } + hints := core.GenHintsFromPhysicalPlan(p) + + // test the new genHints code + flat := core.FlattenPhysicalPlan(p, false) + newHints := core.GenHintsFromFlatPlan(flat) + assertSameHints(t, hints, newHints) + + require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(hints), comment) + } +} + +func TestIndexMergeHint(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + + var input []string + var output []struct { + SQL string + Best string + HasWarn bool + Hints string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + ctx := context.Background() + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + + for i, test := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, test) + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) + stmt, err := p.ParseOneStmt(test, "", "") + require.NoError(t, err, comment) + sctx := tk.Session() + err = executor.ResetContextOfStmt(sctx, stmt) + require.NoError(t, err) + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = test + output[i].Best = core.ToString(p) + output[i].HasWarn = len(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) > 0 + output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) + }) + require.Equal(t, output[i].Best, core.ToString(p), comment) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + if output[i].HasWarn { + require.Len(t, warnings, 1, comment) + } else { + require.Len(t, warnings, 0, comment) + } + hints := core.GenHintsFromPhysicalPlan(p) + + // test the new genHints code + flat := core.FlattenPhysicalPlan(p, false) + newHints := core.GenHintsFromFlatPlan(flat) + assertSameHints(t, hints, newHints) + + require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(hints), comment) + } +} + +func TestQueryBlockHint(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + + var input []string + var output []struct { + SQL string + Plan string + Hints string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + ctx := context.TODO() + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + + for i, tt := range input { + comment := fmt.Sprintf("case:%v sql: %s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err, comment) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = core.ToString(p) + output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) + }) + require.Equal(t, output[i].Plan, core.ToString(p), comment) + hints := core.GenHintsFromPhysicalPlan(p) + + // test the new genHints code + flat := core.FlattenPhysicalPlan(p, false) + newHints := core.GenHintsFromFlatPlan(flat) + assertSameHints(t, hints, newHints) + + require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(hints), comment) + } +} + +func TestInlineProjection(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`drop table if exists test.t1, test.t2;`) + tk.MustExec(`create table test.t1(a bigint, b bigint, index idx_a(a), index idx_b(b));`) + tk.MustExec(`create table test.t2(a bigint, b bigint, index idx_a(a), index idx_b(b));`) + + var input []string + var output []struct { + SQL string + Plan string + Hints string + } + is := domain.GetDomain(tk.Session()).InfoSchema() + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + ctx := context.Background() + p := parser.New() + + for i, tt := range input { + comment := fmt.Sprintf("case:%v sql: %s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err, comment) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = core.ToString(p) + output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) + }) + require.Equal(t, output[i].Plan, core.ToString(p), comment) + hints := core.GenHintsFromPhysicalPlan(p) + + // test the new genHints code + flat := core.FlattenPhysicalPlan(p, false) + newHints := core.GenHintsFromFlatPlan(flat) + assertSameHints(t, hints, newHints) + + require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(hints), comment) + } +} + +func TestIndexJoinHint(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 test.t1, test.t2, test.t;`) + tk.MustExec(`create table test.t1(a bigint, b bigint, index idx_a(a), index idx_b(b));`) + tk.MustExec(`create table test.t2(a bigint, b bigint, index idx_a(a), index idx_b(b));`) + tk.MustExec("CREATE TABLE `t` ( `a` bigint(20) NOT NULL, `b` tinyint(1) DEFAULT NULL, `c` datetime DEFAULT NULL, `d` int(10) unsigned DEFAULT NULL, `e` varchar(20) DEFAULT NULL, `f` double DEFAULT NULL, `g` decimal(30,5) DEFAULT NULL, `h` float DEFAULT NULL, `i` date DEFAULT NULL, `j` timestamp NULL DEFAULT NULL, PRIMARY KEY (`a`), UNIQUE KEY `b` (`b`), KEY `c` (`c`,`d`,`e`), KEY `f` (`f`), KEY `g` (`g`,`h`), KEY `g_2` (`g`), UNIQUE KEY `g_3` (`g`), KEY `i` (`i`) );") + + var input []string + var output []struct { + SQL string + Plan string + } + + is := domain.GetDomain(tk.Session()).InfoSchema() + p := parser.New() + ctx := context.Background() + + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + comment := fmt.Sprintf("case:%v sql: %s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err, comment) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = core.ToString(p) + }) + require.Equal(t, output[i].Plan, core.ToString(p), comment) + } +} + +func doTestDAGPlanBuilderWindow(t *testing.T, vars, input []string, output []struct { + SQL string + Best string +}) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + for _, v := range vars { + tk.MustExec(v) + } + + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + + for i, tt := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + err = sessiontxn.NewTxn(context.Background(), tk.Session()) + require.NoError(t, err) + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Best = core.ToString(p) + }) + require.Equal(t, output[i].Best, core.ToString(p), comment) + } +} + +func TestDAGPlanBuilderWindow(t *testing.T) { + var input []string + var output []struct { + SQL string + Best string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + vars := []string{ + "set @@session.tidb_window_concurrency = 1", + } + doTestDAGPlanBuilderWindow(t, vars, input, output) +} + +func TestDAGPlanBuilderWindowParallel(t *testing.T) { + var input []string + var output []struct { + SQL string + Best string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + vars := []string{ + "set @@session.tidb_window_concurrency = 4", + } + doTestDAGPlanBuilderWindow(t, vars, input, output) +} + +func TestNominalSort(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + var input []string + var output []struct { + SQL string + Plan []string + Result []string + } + tk.MustExec("create table t (a int, b int, index idx_a(a), index idx_b(b))") + tk.MustExec("insert into t values(1, 1)") + tk.MustExec("insert into t values(1, 2)") + tk.MustExec("insert into t values(2, 4)") + tk.MustExec("insert into t values(3, 5)") + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Check(testkit.Rows(output[i].Result...)) + } +} + +func TestHintFromDiffDatabase(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`drop table if exists test.t1`) + tk.MustExec(`create table test.t1(a bigint, index idx_a(a));`) + tk.MustExec(`create table test.t2(a bigint, index idx_a(a));`) + tk.MustExec("drop database if exists test2") + tk.MustExec("create database test2") + tk.MustExec("use test2") + + var input []string + var output []struct { + SQL string + Plan string + } + is := domain.GetDomain(tk.Session()).InfoSchema() + p := parser.New() + ctx := context.Background() + + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + comment := fmt.Sprintf("case:%v sql: %s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err, comment) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = core.ToString(p) + }) + require.Equal(t, output[i].Plan, core.ToString(p), comment) + } +} + +func TestNthPlanHintWithExplain(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`drop table if exists test.tt`) + tk.MustExec(`create table test.tt (a int,b int, index(a), index(b));`) + tk.MustExec("insert into tt values (1, 1), (2, 2), (3, 4)") + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) + + var input []string + var output []struct { + SQL string + Plan []string + } + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + } + + // This assertion makes sure a query with or without nth_plan() hint output exactly the same plan(including plan ID). + // The query below is the same as queries in the testdata except for nth_plan() hint. + // Currently, its output is the same as the second test case in the testdata, which is `output[1]`. If this doesn't + // hold in the future, you may need to modify this. + tk.MustQuery("explain format = 'brief' select * from test.tt where a=1 and b=1").Check(testkit.Rows(output[1].Plan...)) +} + +func TestEnumIndex(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + 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(e enum('c','b','a',''), index idx(e))") + tk.MustExec("insert ignore into t values(0),(1),(2),(3),(4);") + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + }) + tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) + } +} + +func TestIssue27233(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("CREATE TABLE `PK_S_MULTI_31` (\n `COL1` tinyint(45) NOT NULL,\n `COL2` tinyint(45) NOT NULL,\n PRIMARY KEY (`COL1`,`COL2`) /*T![clustered_index] NONCLUSTERED */\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;") + tk.MustExec("insert into PK_S_MULTI_31 values(122,100),(124,-22),(124,34),(127,103);") + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + }) + tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) + } +} + +func TestSelectionPartialPushDown(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + 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 as (a+1) virtual)") + tk.MustExec("create table t2(a int, b int as (a+1) virtual, c int, key idx_a(a))") + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) + }) + tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestIssue28316(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + 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)") + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) + }) + tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestSkewDistinctAgg(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + 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(11), `b` int(11), `c` int(11), `d` date)") + tk.MustExec("insert into t (a,b,c,d) value(1,4,5,'2019-06-01')") + tk.MustExec("insert into t (a,b,c,d) value(2,null,1,'2019-07-01')") + tk.MustExec("insert into t (a,b,c,d) value(3,4,5,'2019-08-01')") + tk.MustExec("insert into t (a,b,c,d) value(3,6,2,'2019-09-01')") + tk.MustExec("insert into t (a,b,c,d) value(10,4,null,'2020-06-01')") + tk.MustExec("insert into t (a,b,c,d) value(20,null,1,'2020-07-01')") + tk.MustExec("insert into t (a,b,c,d) value(30,4,5,'2020-08-01')") + tk.MustExec("insert into t (a,b,c,d) value(30,6,5,'2020-09-01')") + tk.MustQuery("select date_format(d,'%Y') as df, sum(a), count(b), count(distinct c) " + + "from t group by date_format(d,'%Y') order by df;").Check( + testkit.Rows("2019 9 3 3", "2020 90 3 2")) + tk.MustExec("set @@tidb_opt_skew_distinct_agg=1") + tk.MustQuery("select date_format(d,'%Y') as df, sum(a), count(b), count(distinct c) " + + "from t group by date_format(d,'%Y') order by df;").Check( + testkit.Rows("2019 9 3 3", "2020 90 3 2")) + tk.MustQuery("select count(distinct b), sum(c) from t group by a order by 1,2;").Check( + testkit.Rows("0 1", "0 1", "1 ", "1 5", "2 7", "2 10")) + tk.MustQuery("select count(distinct b) from t group by date_format(d,'%Y') order by 1;").Check( + testkit.Rows("2", "2")) + tk.MustQuery("select count(a), count(distinct b), max(b) from t group by date_format(d,'%Y') order by 1,2,3;").Check( + testkit.Rows("4 2 6", "4 2 6")) + tk.MustQuery("select count(a), count(distinct b), max(b) from t group by date_format(d,'%Y'),c order by 1,2,3;").Check( + testkit.Rows("1 0 ", "1 0 ", "1 1 4", "1 1 6", "2 1 4", "2 2 6")) + tk.MustQuery("select avg(distinct b), count(a), sum(b) from t group by date_format(d,'%Y'),c order by 1,2,3;").Check( + testkit.Rows(" 1 ", " 1 ", "4.0000 1 4", "4.0000 2 8", "5.0000 2 10", "6.0000 1 6")) + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + ts).Rows()) + }) + tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestHJBuildAndProbeHint(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + Warning []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + 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(a int primary key, b int not null)") + tk.MustExec("create table t2(a int primary key, b int not null)") + tk.MustExec("create table t3(a int primary key, b int not null)") + tk.MustExec("insert into t1 values(1,1),(2,2)") + tk.MustExec("insert into t2 values(1,1),(2,1)") + tk.MustExec("insert into t3 values(1,1),(2,1)") + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + output[i].Warning = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) + tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warning...)) + } +} + +func TestHJBuildAndProbeHint4StaticPartitionTable(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + Warning []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + 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(a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table t2(a int, b int) partition by hash(a) partitions 5`) + tk.MustExec(`create table t3(a int, b int) partition by hash(b) partitions 3`) + tk.MustExec("insert into t1 values(1,1),(2,2)") + tk.MustExec("insert into t2 values(1,1),(2,1)") + tk.MustExec("insert into t3 values(1,1),(2,1)") + tk.MustExec(`set @@tidb_partition_prune_mode="static"`) + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + output[i].Warning = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) + } +} + +func TestHJBuildAndProbeHint4DynamicPartitionTable(t *testing.T) { + failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) + defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") + + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + Warning []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2, t3") + tk.MustExec(`create table t1(a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table t2(a int, b int) partition by hash(a) partitions 5`) + tk.MustExec(`create table t3(a int, b int) partition by hash(b) partitions 3`) + tk.MustExec("insert into t1 values(1,1),(2,2)") + tk.MustExec("insert into t2 values(1,1),(2,1)") + tk.MustExec("insert into t3 values(1,1),(2,1)") + tk.MustExec(`set @@tidb_partition_prune_mode="dynamic"`) + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + output[i].Warning = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) + } +} + +func TestHJBuildAndProbeHint4TiFlash(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Warning []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + 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(a int primary key, b int not null)") + tk.MustExec("create table t2(a int primary key, b int not null)") + tk.MustExec("create table t3(a int primary key, b int not null)") + tk.MustExec("insert into t1 values(1,1),(2,2)") + tk.MustExec("insert into t2 values(1,1),(2,1)") + tk.MustExec("insert into t3 values(1,1),(2,1)") + // 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 { + tableName := tblInfo.Name.L + if tableName == "t1" || tableName == "t2" || tableName == "t3" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Warning = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestMPPSinglePartitionType(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists employee") + tk.MustExec("create table employee(empid int, deptid int, salary decimal(10,2))") + tk.MustExec("set tidb_enforce_mpp=0") + + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "employee" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + }) + if strings.HasPrefix(ts, "set") { + tk.MustExec(ts) + continue + } + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) + }) + tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestNoDecorrelateHint(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + Warning []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + 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)") + tk.MustExec("create table t2(a int primary key, b int)") + tk.MustExec("create table t3(a int, b int)") + tk.MustExec("insert into t1 values(1,1),(2,2)") + tk.MustExec("insert into t2 values(1,1),(2,1)") + tk.MustExec("insert into t3 values(1,1),(2,1)") + + tk.MustExec("create table ta(id int, code int, name varchar(20), index idx_ta_id(id),index idx_ta_name(name), index idx_ta_code(code))") + tk.MustExec("create table tb(id int, code int, name varchar(20), index idx_tb_id(id),index idx_tb_name(name))") + tk.MustExec("create table tc(id int, code int, name varchar(20), index idx_tc_id(id),index idx_tc_name(name))") + tk.MustExec("create table td(id int, code int, name varchar(20), index idx_tc_id(id),index idx_tc_name(name))") + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + output[i].Warning = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) + tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warning...)) + } +} + +func TestCountStarForTikv(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Warning []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=1") + tk.MustExec("create table t (a int(11) not null, b varchar(10) not null, c date not null, d char(1) not null, e bigint not null, f datetime not null, g bool not null, h bool )") + tk.MustExec("create table t_pick_row_id (a char(20) not null)") + + // tikv + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestCountStarForTiFlash(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Warning []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=1") + tk.MustExec("create table t (a int(11) not null, b varchar(10) not null, c date not null, d char(1) not null, e bigint not null, f datetime not null, g bool not null, h bool )") + tk.MustExec("create table t_pick_row_id (a char(20) not null)") + + // tiflash + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + tableName := tblInfo.Name.L + if tableName == "t" || tableName == "t_pick_row_id" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestHashAggPushdownToTiFlashCompute(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + Warning []string + } + ) + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists tbl_15;") + tk.MustExec(`create table tbl_15 (col_89 text (473) collate utf8mb4_bin , + col_90 timestamp default '1976-04-03' , + col_91 tinyint unsigned not null , + col_92 tinyint , + col_93 double not null , + col_94 datetime not null default '1970-06-08' , + col_95 datetime default '2028-02-13' , + col_96 int unsigned not null default 2532480521 , + col_97 char (168) default '') partition by hash (col_91) partitions 4;`) + + tk.MustExec("drop table if exists tbl_16;") + tk.MustExec(`create table tbl_16 (col_98 text (246) not null , + col_99 decimal (30 ,19) , + col_100 mediumint unsigned , + col_101 text (410) collate utf8mb4_bin , + col_102 date not null , + col_103 timestamp not null default '2003-08-27' , + col_104 text (391) not null , + col_105 date default '2010-10-24' , + col_106 text (9) not null,primary key (col_100, col_98(5), col_103), + unique key idx_23 (col_100, col_106 (3), col_101 (3))) partition by hash (col_100) partitions 2;`) + + config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = true + }) + defer config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = false + }) + + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + tableName := tblInfo.Name.L + if tableName == "tbl_15" || tableName == "tbl_16" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + tk.MustExec("set @@tidb_partition_prune_mode = 'static';") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash';") + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/planner/core/integration_partition_test.go b/planner/core/integration_partition_test.go index b17dcf6c72812..4d031c87553e8 100644 --- a/planner/core/integration_partition_test.go +++ b/planner/core/integration_partition_test.go @@ -23,173 +23,14 @@ import ( "strings" "testing" - "github.com/pingcap/failpoint" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/auth" - "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/util/benchdaily" "github.com/stretchr/testify/require" ) -func TestListPartitionPushDown(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("create database list_push_down") - tk.MustExec("use list_push_down") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists tlist") - tk.MustExec(`set tidb_enable_list_partition = 1`) - tk.MustExec(`create table tlist (a int) partition by list (a) ( - partition p0 values in (0, 1, 2), - partition p1 values in (3, 4, 5))`) - tk.MustExec(`create table tcollist (a int) partition by list columns(a) ( - partition p0 values in (0, 1, 2), - partition p1 values in (3, 4, 5))`) - tk.MustExec("set @@tidb_partition_prune_mode = 'static'") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationPartitionSuiteData := core.GetIntegrationPartitionSuiteData() - integrationPartitionSuiteData.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 TestListColVariousTypes(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("create database list_col_partition_types") - tk.MustExec("use list_col_partition_types") - tk.MustExec("drop table if exists tlist") - tk.MustExec(`set tidb_enable_list_partition = 1`) - - tk.MustExec(`create table tint (a int) partition by list columns(a) (partition p0 values in (0, 1), partition p1 values in (2, 3))`) - tk.MustExec(`create table tdate (a date) partition by list columns(a) (partition p0 values in ('2000-01-01', '2000-01-02'), partition p1 values in ('2000-01-03', '2000-01-04'))`) - tk.MustExec(`create table tstring (a varchar(32)) partition by list columns(a) (partition p0 values in ('a', 'b'), partition p1 values in ('c', 'd'))`) - - err := tk.ExecToErr(`create table tdouble (a double) partition by list columns(a) (partition p0 values in (0, 1), partition p1 values in (2, 3))`) - require.Error(t, err) - require.Contains(t, err.Error(), "not allowed") - - err = tk.ExecToErr(`create table tdecimal (a decimal(30, 10)) partition by list columns(a) (partition p0 values in (0, 1), partition p1 values in (2, 3))`) - require.Error(t, err) - require.Contains(t, err.Error(), "not allowed") - - tk.MustExec(`insert into tint values (0), (1), (2), (3)`) - tk.MustExec(`insert into tdate values ('2000-01-01'), ('2000-01-02'), ('2000-01-03'), ('2000-01-04')`) - tk.MustExec(`insert into tstring values ('a'), ('b'), ('c'), ('d')`) - tk.MustExec(`analyze table tint`) - tk.MustExec(`analyze table tdate`) - tk.MustExec(`analyze table tstring`) - - var input []string - var output []struct { - SQL string - Results []string - } - integrationPartitionSuiteData := core.GetIntegrationPartitionSuiteData() - integrationPartitionSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Results = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Results...)) - } -} - -func TestListPartitionPruning(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("create database list_partition_pruning") - tk.MustExec("use list_partition_pruning") - tk.MustExec("drop table if exists tlist") - tk.MustExec(`set tidb_enable_list_partition = 1`) - tk.MustExec(`create table tlist (a int) partition by list (a) ( - partition p0 values in (0, 1, 2), - partition p1 values in (3, 4, 5), - partition p2 values in (6, 7, 8), - partition p3 values in (9, 10, 11))`) - tk.MustExec(`create table tcollist (a int) partition by list columns(a) ( - partition p0 values in (0, 1, 2), - partition p1 values in (3, 4, 5), - partition p2 values in (6, 7, 8), - partition p3 values in (9, 10, 11))`) - tk.MustExec(`analyze table tlist`) - tk.MustExec(`analyze table tcollist`) - - var input []string - var output []struct { - SQL string - DynamicPlan []string - StaticPlan []string - } - integrationPartitionSuiteData := core.GetIntegrationPartitionSuiteData() - integrationPartitionSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") - output[i].DynamicPlan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - tk.MustExec("set @@tidb_partition_prune_mode = 'static'") - output[i].StaticPlan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") - tk.MustQuery(tt).Check(testkit.Rows(output[i].DynamicPlan...)) - tk.MustExec("set @@tidb_partition_prune_mode = 'static'") - tk.MustQuery(tt).Check(testkit.Rows(output[i].StaticPlan...)) - } -} - -func TestListPartitionFunctions(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("create database list_partition_pruning") - tk.MustExec("use list_partition_pruning") - tk.MustExec("set tidb_enable_list_partition = 1") - tk.MustExec("set @@tidb_partition_prune_mode = 'static'") - - var input []string - var output []struct { - SQL string - Results []string - } - integrationPartitionSuiteData := core.GetIntegrationPartitionSuiteData() - integrationPartitionSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Results = nil - if strings.Contains(tt, "select") { - output[i].Results = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) - } - }) - - if strings.Contains(tt, "select") { - tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Results...)) - } else { - tk.MustExec(tt) - } - } -} - func TestListPartitionOrderLimit(t *testing.T) { store := testkit.CreateMockStore(t) @@ -1655,43 +1496,3 @@ func TestPartitionProcessorWithUninitializedTable(t *testing.T) { } tk.MustQuery("explain format=brief select * from q1,q2").CheckAt([]int{0}, rows) } - -func TestEstimationForTopNPushToDynamicPartition(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 tlist") - tk.MustExec(`set tidb_enable_list_partition = 1`) - tk.MustExec(`create table trange (a int, b int, c int, index ia(a), primary key (b) clustered) - partition by range(b) ( - partition p1 values less than(100), - partition p2 values less than(200), - partition p3 values less than maxvalue);`) - tk.MustExec(`create table tlist (a int, b int, c int, index ia(a), primary key (b) clustered) - partition by list (b) ( - partition p0 values in (0, 1, 2), - partition p1 values in (3, 4, 5));`) - tk.MustExec(`create table thash (a int, b int, c int, index ia(a), primary key (b) clustered) - partition by hash(b) partitions 4;`) - tk.MustExec(`create table t (a int, b int, c int, index ia(a), primary key (b) clustered);`) - tk.MustExec(`analyze table trange;`) - tk.MustExec(`analyze table tlist;`) - tk.MustExec(`analyze table thash;`) - tk.MustExec(`analyze table t;`) - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationPartitionSuiteData := core.GetIntegrationPartitionSuiteData() - integrationPartitionSuiteData.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...)) - } -} diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 1664fdd098421..c01cf4102cf05 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -163,85 +163,6 @@ func TestAggPushDownLeftJoin(t *testing.T) { " └─TableFullScan 10000.00 cop[tikv] table:customer keep order:false, stats:pseudo")) } -func TestPushLimitDownIndexLookUpReader(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("set @@session.tidb_executor_concurrency = 4;") - tk.MustExec("set @@session.tidb_hash_join_concurrency = 5;") - tk.MustExec("set @@session.tidb_distsql_scan_concurrency = 15;") - tk.MustExec("use test") - tk.MustExec("drop table if exists tbl") - tk.MustExec("create table tbl(a int, b int, c int, key idx_b_c(b,c))") - tk.MustExec("insert into tbl values(1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5)") - tk.MustExec("analyze table tbl") - - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -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 := core.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 := 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(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestIssue22298(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -377,31 +298,6 @@ func TestAntiJoinConstProp(t *testing.T) { )) } -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 := 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(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestNoneAccessPathsFoundByIsolationRead(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -432,10 +328,50 @@ func TestNoneAccessPathsFoundByIsolationRead(t *testing.T) { tk.MustExec("select * from t") } -func TestSelPushDownTiFlash(t *testing.T) { +func TestPartitionTableDynamicModeUnderNewCollation(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database test_new_collation") + tk.MustExec("use test_new_collation") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // hash + range partition + tk.MustExec(`CREATE TABLE thash (a int, c varchar(20) charset utf8mb4 collate utf8mb4_general_ci, key(a)) partition by hash(a) partitions 4`) + tk.MustExec(`CREATE TABLE trange (a int, c varchar(20) charset utf8mb4 collate utf8mb4_general_ci, key(a)) partition by range(a) ( + partition p0 values less than (10), + partition p1 values less than (20), + partition p2 values less than (30), + partition p3 values less than (40))`) + tk.MustExec(`insert into thash values (1, 'a'), (1, 'A'), (11, 'a'), (11, 'A'), (21, 'a'), (21, 'A'), (31, 'a'), (31, 'A')`) + tk.MustExec(`insert into trange values (1, 'a'), (1, 'A'), (11, 'a'), (11, 'A'), (21, 'a'), (21, 'A'), (31, 'a'), (31, 'A')`) + tk.MustQuery(`select * from thash use index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) + tk.MustQuery(`select * from thash ignore index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) + tk.MustQuery(`select * from trange use index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) + tk.MustQuery(`select * from trange ignore index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) + + // range partition and partitioned by utf8mb4_general_ci + tk.MustExec(`create table strrange(a varchar(10) charset utf8mb4 collate utf8mb4_general_ci, b int) partition by range columns(a) ( + partition p0 values less than ('a'), + partition p1 values less than ('k'), + partition p2 values less than ('z'))`) + tk.MustExec("insert into strrange values ('a', 1), ('A', 1), ('y', 1), ('Y', 1), ('q', 1)") + tk.MustQuery("select * from strrange where a in ('a', 'y')").Sort().Check(testkit.Rows("A 1", "Y 1", "a 1", "y 1")) + + // list partition and partitioned by utf8mb4_general_ci + tk.MustExec(`create table strlist(a varchar(10) charset utf8mb4 collate utf8mb4_general_ci, b int) partition by list columns (a) ( + partition p0 values in ('a', 'b'), + partition p1 values in ('c', 'd'), + partition p2 values in ('e', 'f'))`) + tk.MustExec("insert into strlist values ('a', 1), ('A', 1), ('d', 1), ('D', 1), ('e', 1)") + tk.MustQuery(`select * from strlist where a='a'`).Sort().Check(testkit.Rows("A 1", "a 1")) + tk.MustQuery(`select * from strlist where a in ('D', 'e')`).Sort().Check(testkit.Rows("D 1", "d 1", "e 1")) +} + +func TestAggPushDownEngine(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 primary key, b varchar(20))") @@ -454,51 +390,38 @@ func TestSelPushDownTiFlash(t *testing.T) { } tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_allow_mpp = 0") - tk.MustExec("set tidb_cost_model_version=2") - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } + tk.MustQuery("explain format = 'brief' select approx_count_distinct(a) from t").Check(testkit.Rows( + "StreamAgg 1.00 root funcs:approx_count_distinct(Column#5)->Column#3", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:approx_count_distinct(test.t.a)->Column#5", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tikv'") + + tk.MustQuery("explain format = 'brief' select approx_count_distinct(a) from t").Check(testkit.Rows( + "HashAgg 1.00 root funcs:approx_count_distinct(test.t.a)->Column#3", + "└─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo")) } -func TestVerboseExplain(t *testing.T) { +func TestIssue15110(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") + tk.MustExec("drop table if exists crm_rd_150m") + tk.MustExec(`CREATE TABLE crm_rd_150m ( + product varchar(256) DEFAULT NULL, + uks varchar(16) DEFAULT NULL, + brand varchar(256) DEFAULT NULL, + cin varchar(16) DEFAULT NULL, + created_date timestamp NULL DEFAULT NULL, + quantity int(11) DEFAULT NULL, + amount decimal(11,0) DEFAULT NULL, + pl_date timestamp NULL DEFAULT NULL, + customer_first_date timestamp NULL DEFAULT NULL, + recent_date timestamp NULL DEFAULT NULL + ) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin;`) // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) @@ -506,7 +429,7 @@ func TestVerboseExplain(t *testing.T) { 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" { + if tblInfo.Name.L == "crm_rd_150m" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, Available: true, @@ -514,894 +437,39 @@ func TestVerboseExplain(t *testing.T) { } } - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("explain format = 'brief' SELECT count(*) FROM crm_rd_150m dataset_48 WHERE (CASE WHEN (month(dataset_48.customer_first_date)) <= 30 THEN '新客' ELSE NULL END) IS NOT NULL;") } -func TestPushDownToTiFlashWithKeepOrder(t *testing.T) { +func TestIssue40910(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 varchar(20))") + tk.MustExec(`create table t(a int, b int, index idx_a(a), index idx_b(b));`) - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } + tk.MustExec("select * from t where a > 1 and a < 10 order by b;") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) + tk.MustExec("create session binding for select * from t where a > 1 and a < 10 order by b using select /*+ use_index(t, idx_a) */ * from t where a > 1 and a < 10 order by b;") + tk.MustExec("select * from t where a > 1 and a < 10 order by b;") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_allow_mpp = 0") - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } + tk.MustExec("select /*+ use_index(t, idx_b) */ * from t where a > 1 and a < 10 order by b;") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + + tk.MustExec("select /*+ use_index(t, idx_b) */ * from t where a > 1 and a < 10 order by b;") + tk.MustQuery("show warnings").Check(testkit.Rows( + "Warning 1105 The system ignores the hints in the current query and uses the hints specified in the bindSQL: SELECT /*+ use_index(`t` `idx_a`)*/ * FROM `test`.`t` WHERE `a` > 1 AND `a` < 10 ORDER BY `b`")) } -func TestPushDownToTiFlashWithKeepOrderInFastMode(t *testing.T) { +func TestKeepOrderHintWithBinding(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 varchar(20))") - tk.MustExec("set @@session.tiflash_fastscan=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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_allow_mpp = 0") - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestMPPJoin(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 d1_t") - tk.MustExec("create table d1_t(d1_k int, value int)") - tk.MustExec("insert into d1_t values(1,2),(2,3)") - tk.MustExec("analyze table d1_t") - tk.MustExec("drop table if exists d2_t") - tk.MustExec("create table d2_t(d2_k decimal(10,2), value int)") - tk.MustExec("insert into d2_t values(10.11,2),(10.12,3)") - tk.MustExec("analyze table d2_t") - tk.MustExec("drop table if exists d3_t") - tk.MustExec("create table d3_t(d3_k date, value int)") - tk.MustExec("insert into d3_t values(date'2010-01-01',2),(date'2010-01-02',3)") - tk.MustExec("analyze table d3_t") - tk.MustExec("drop table if exists fact_t") - tk.MustExec("create table fact_t(d1_k int, d2_k decimal(10,2), d3_k date, col1 int, col2 int, col3 int)") - tk.MustExec("insert into fact_t values(1,10.11,date'2010-01-01',1,2,3),(1,10.11,date'2010-01-02',1,2,3),(1,10.12,date'2010-01-01',1,2,3),(1,10.12,date'2010-01-02',1,2,3)") - tk.MustExec("insert into fact_t values(2,10.11,date'2010-01-01',1,2,3),(2,10.11,date'2010-01-02',1,2,3),(2,10.12,date'2010-01-01',1,2,3),(2,10.12,date'2010-01-02',1,2,3)") - tk.MustExec("analyze table fact_t") - - // 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 == "fact_t" || tblInfo.Name.L == "d1_t" || tblInfo.Name.L == "d2_t" || tblInfo.Name.L == "d3_t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_allow_mpp = 1") - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestMPPLeftSemiJoin(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - // test table - tk.MustExec("use test") - tk.MustExec("create table test.t(a int not null, b int null);") - tk.MustExec("set tidb_allow_mpp=1; set tidb_enforce_mpp=1;") - - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - var input []string - var output []struct { - SQL string - Plan []string - Warn []string - } - integrationSuiteData := core.GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - }) - if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { - tk.MustExec(tt) - continue - } - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} - -func TestMPPOuterJoinBuildSideForBroadcastJoin(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 a") - tk.MustExec("create table a(id int, value int)") - tk.MustExec("insert into a values(1,2),(2,3)") - tk.MustExec("analyze table a") - tk.MustExec("drop table if exists b") - tk.MustExec("create table b(id int, value int)") - tk.MustExec("insert into b values(1,2),(2,3),(3,4)") - tk.MustExec("analyze table b") - // 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 == "a" || tblInfo.Name.L == "b" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 0") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 10000") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 10000") - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestMPPOuterJoinBuildSideForShuffleJoinWithFixedBuildSide(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 a") - tk.MustExec("create table a(id int, value int)") - tk.MustExec("insert into a values(1,2),(2,3)") - tk.MustExec("analyze table a") - tk.MustExec("drop table if exists b") - tk.MustExec("create table b(id int, value int)") - tk.MustExec("insert into b values(1,2),(2,3),(3,4)") - tk.MustExec("analyze table b") - // 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 == "a" || tblInfo.Name.L == "b" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 1") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 0") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 0") - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestMPPOuterJoinBuildSideForShuffleJoin(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 a") - tk.MustExec("create table a(id int, value int)") - tk.MustExec("insert into a values(1,2),(2,3)") - tk.MustExec("analyze table a") - tk.MustExec("drop table if exists b") - tk.MustExec("create table b(id int, value int)") - tk.MustExec("insert into b values(1,2),(2,3),(3,4)") - tk.MustExec("analyze table b") - // 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 == "a" || tblInfo.Name.L == "b" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 0") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 0") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 0") - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestMPPShuffledJoin(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 d1_t") - tk.MustExec("create table d1_t(d1_k int, value int)") - tk.MustExec("insert into d1_t values(1,2),(2,3)") - tk.MustExec("insert into d1_t values(1,2),(2,3)") - tk.MustExec("analyze table d1_t") - tk.MustExec("drop table if exists d2_t") - tk.MustExec("create table d2_t(d2_k decimal(10,2), value int)") - tk.MustExec("insert into d2_t values(10.11,2),(10.12,3)") - tk.MustExec("insert into d2_t values(10.11,2),(10.12,3)") - tk.MustExec("analyze table d2_t") - tk.MustExec("drop table if exists d3_t") - tk.MustExec("create table d3_t(d3_k date, value int)") - tk.MustExec("insert into d3_t values(date'2010-01-01',2),(date'2010-01-02',3)") - tk.MustExec("insert into d3_t values(date'2010-01-01',2),(date'2010-01-02',3)") - tk.MustExec("analyze table d3_t") - tk.MustExec("drop table if exists fact_t") - tk.MustExec("create table fact_t(d1_k int, d2_k decimal(10,2), d3_k date, col1 int, col2 int, col3 int)") - tk.MustExec("insert into fact_t values(1,10.11,date'2010-01-01',1,2,3),(1,10.11,date'2010-01-02',1,2,3),(1,10.12,date'2010-01-01',1,2,3),(1,10.12,date'2010-01-02',1,2,3)") - tk.MustExec("insert into fact_t values(2,10.11,date'2010-01-01',1,2,3),(2,10.11,date'2010-01-02',1,2,3),(2,10.12,date'2010-01-01',1,2,3),(2,10.12,date'2010-01-02',1,2,3)") - tk.MustExec("insert into fact_t values(2,10.11,date'2010-01-01',1,2,3),(2,10.11,date'2010-01-02',1,2,3),(2,10.12,date'2010-01-01',1,2,3),(2,10.12,date'2010-01-02',1,2,3)") - tk.MustExec("insert into fact_t values(2,10.11,date'2010-01-01',1,2,3),(2,10.11,date'2010-01-02',1,2,3),(2,10.12,date'2010-01-01',1,2,3),(2,10.12,date'2010-01-02',1,2,3)") - tk.MustExec("analyze table fact_t") - - // 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 == "fact_t" || tblInfo.Name.L == "d1_t" || tblInfo.Name.L == "d2_t" || tblInfo.Name.L == "d3_t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_allow_mpp = 1") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 1") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 1") - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestMPPJoinWithCanNotFoundColumnInSchemaColumnsError(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") - tk.MustExec("create table t1(id int, v1 decimal(20,2), v2 decimal(20,2))") - tk.MustExec("create table t2(id int, v1 decimal(10,2), v2 decimal(10,2))") - tk.MustExec("create table t3(id int, v1 decimal(10,2), v2 decimal(10,2))") - tk.MustExec("insert into t1 values(1,1,1),(2,2,2)") - tk.MustExec("insert into t2 values(1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8)") - tk.MustExec("insert into t3 values(1,1,1)") - tk.MustExec("analyze table t1") - tk.MustExec("analyze table t2") - tk.MustExec("analyze table t3") - - 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.Name.L == "t3" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_enforce_mpp = 1") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 0") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 0") - tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 0") - - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestJoinNotSupportedByTiFlash(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists table_1") - tk.MustExec("create table table_1(id int not null, bit_col bit(2) not null, datetime_col datetime not null)") - tk.MustExec("insert into table_1 values(1,b'1','2020-01-01 00:00:00'),(2,b'0','2020-01-01 00:00:00')") - tk.MustExec("analyze table table_1") - - tk.MustExec("insert into mysql.expr_pushdown_blacklist values('dayofmonth', 'tiflash', '');") - tk.MustExec("admin reload expr_pushdown_blacklist;") - - // 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 == "table_1" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_allow_mpp = 1") - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } - - tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 1") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 1") - 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 TestMPPWithHashExchangeUnderNewCollation(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists table_1") - tk.MustExec("create table table_1(id int not null, value char(10)) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci;") - tk.MustExec("insert into table_1 values(1,'1'),(2,'2')") - tk.MustExec("drop table if exists table_2") - tk.MustExec("create table table_2(id int not null, value char(10)) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin;") - tk.MustExec("insert into table_2 values(1,'1'),(2,'2')") - tk.MustExec("analyze table table_1") - tk.MustExec("analyze table table_2") - - // 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 == "table_1" || tblInfo.Name.L == "table_2" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_allow_mpp = 1") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 0") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 0") - tk.MustExec("set @@session.tidb_hash_exchange_with_new_collation = 1") - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestMPPWithBroadcastExchangeUnderNewCollation(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists table_1") - tk.MustExec("create table table_1(id int not null, value char(10))") - tk.MustExec("insert into table_1 values(1,'1'),(2,'2')") - tk.MustExec("analyze table table_1") - - // 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 == "table_1" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_allow_mpp = 1") - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestPartitionTableDynamicModeUnderNewCollation(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("create database test_new_collation") - tk.MustExec("use test_new_collation") - tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") - - // hash + range partition - tk.MustExec(`CREATE TABLE thash (a int, c varchar(20) charset utf8mb4 collate utf8mb4_general_ci, key(a)) partition by hash(a) partitions 4`) - tk.MustExec(`CREATE TABLE trange (a int, c varchar(20) charset utf8mb4 collate utf8mb4_general_ci, key(a)) partition by range(a) ( - partition p0 values less than (10), - partition p1 values less than (20), - partition p2 values less than (30), - partition p3 values less than (40))`) - tk.MustExec(`insert into thash values (1, 'a'), (1, 'A'), (11, 'a'), (11, 'A'), (21, 'a'), (21, 'A'), (31, 'a'), (31, 'A')`) - tk.MustExec(`insert into trange values (1, 'a'), (1, 'A'), (11, 'a'), (11, 'A'), (21, 'a'), (21, 'A'), (31, 'a'), (31, 'A')`) - tk.MustQuery(`select * from thash use index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) - tk.MustQuery(`select * from thash ignore index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) - tk.MustQuery(`select * from trange use index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) - tk.MustQuery(`select * from trange ignore index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) - - // range partition and partitioned by utf8mb4_general_ci - tk.MustExec(`create table strrange(a varchar(10) charset utf8mb4 collate utf8mb4_general_ci, b int) partition by range columns(a) ( - partition p0 values less than ('a'), - partition p1 values less than ('k'), - partition p2 values less than ('z'))`) - tk.MustExec("insert into strrange values ('a', 1), ('A', 1), ('y', 1), ('Y', 1), ('q', 1)") - tk.MustQuery("select * from strrange where a in ('a', 'y')").Sort().Check(testkit.Rows("A 1", "Y 1", "a 1", "y 1")) - - // list partition and partitioned by utf8mb4_general_ci - tk.MustExec(`create table strlist(a varchar(10) charset utf8mb4 collate utf8mb4_general_ci, b int) partition by list columns (a) ( - partition p0 values in ('a', 'b'), - partition p1 values in ('c', 'd'), - partition p2 values in ('e', 'f'))`) - tk.MustExec("insert into strlist values ('a', 1), ('A', 1), ('d', 1), ('D', 1), ('e', 1)") - tk.MustQuery(`select * from strlist where a='a'`).Sort().Check(testkit.Rows("A 1", "a 1")) - tk.MustQuery(`select * from strlist where a in ('D', 'e')`).Sort().Check(testkit.Rows("D 1", "d 1", "e 1")) -} - -func TestMPPAvgRewrite(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists table_1") - tk.MustExec("create table table_1(id int not null, value decimal(10,2))") - tk.MustExec("insert into table_1 values(1,1),(2,2)") - tk.MustExec("analyze table table_1") - - // 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 == "table_1" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_allow_mpp = 1") - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestAggPushDownEngine(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 primary key, b varchar(20))") - - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - - tk.MustQuery("explain format = 'brief' select approx_count_distinct(a) from t").Check(testkit.Rows( - "StreamAgg 1.00 root funcs:approx_count_distinct(Column#5)->Column#3", - "└─TableReader 1.00 root data:StreamAgg", - " └─StreamAgg 1.00 batchCop[tiflash] funcs:approx_count_distinct(test.t.a)->Column#5", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) - - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tikv'") - - tk.MustQuery("explain format = 'brief' select approx_count_distinct(a) from t").Check(testkit.Rows( - "HashAgg 1.00 root funcs:approx_count_distinct(test.t.a)->Column#3", - "└─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo")) -} - -func TestIssue15110(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists crm_rd_150m") - tk.MustExec(`CREATE TABLE crm_rd_150m ( - product varchar(256) DEFAULT NULL, - uks varchar(16) DEFAULT NULL, - brand varchar(256) DEFAULT NULL, - cin varchar(16) DEFAULT NULL, - created_date timestamp NULL DEFAULT NULL, - quantity int(11) DEFAULT NULL, - amount decimal(11,0) DEFAULT NULL, - pl_date timestamp NULL DEFAULT NULL, - customer_first_date timestamp NULL DEFAULT NULL, - recent_date timestamp NULL DEFAULT NULL - ) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin;`) - - // 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 == "crm_rd_150m" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("explain format = 'brief' SELECT count(*) FROM crm_rd_150m dataset_48 WHERE (CASE WHEN (month(dataset_48.customer_first_date)) <= 30 THEN '新客' ELSE NULL END) IS NOT NULL;") -} - -func TestIssue40910(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, index idx_a(a), index idx_b(b));`) - - tk.MustExec("select * from t where a > 1 and a < 10 order by b;") - tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) - tk.MustExec("create session binding for select * from t where a > 1 and a < 10 order by b using select /*+ use_index(t, idx_a) */ * from t where a > 1 and a < 10 order by b;") - tk.MustExec("select * from t where a > 1 and a < 10 order by b;") - tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) - - tk.MustExec("select /*+ use_index(t, idx_b) */ * from t where a > 1 and a < 10 order by b;") - tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) - - tk.MustExec("select /*+ use_index(t, idx_b) */ * from t where a > 1 and a < 10 order by b;") - tk.MustQuery("show warnings").Check(testkit.Rows( - "Warning 1105 The system ignores the hints in the current query and uses the hints specified in the bindSQL: SELECT /*+ use_index(`t` `idx_a`)*/ * FROM `test`.`t` WHERE `a` > 1 AND `a` < 10 ORDER BY `b`")) -} - -func TestReadFromStorageHint(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, tt, ttt") - tk.MustExec("set session tidb_allow_mpp=OFF") - tk.MustExec("create table t(a int, b int, index ia(a))") - tk.MustExec("create table tt(a int, b int, primary key(a))") - tk.MustExec("create table ttt(a int, primary key (a desc))") - - // 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, - } - } - - var input []string - var output []struct { - SQL string - Plan []string - Warn []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(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} - -func TestKeepOrderHint(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, t1, th") - tk.MustExec("drop view if exists v, v1") - tk.MustExec("create table t(a int, b int, primary key(a));") - tk.MustExec("create table t1(a int, b int, index idx_a(a));") - tk.MustExec("create table th (a int, key(a)) partition by hash(a) partitions 4;") - tk.MustExec("create definer='root'@'localhost' view v as select * from t1 where a<10 order by a limit 1;") - tk.MustExec("create definer='root'@'localhost' view v1 as select * from t where a<10 order by a limit 1;") - - // If the optimizer can not generate the keep order plan, it will report error - err := tk.ExecToErr("explain select /*+ order_index(t1, idx_a) */ * from t1 where a<10 limit 1;") - require.EqualError(t, err, "[planner:1815]Internal : Can't find a proper physical plan for this query") - - err = tk.ExecToErr("explain select /*+ order_index(t, primary) */ * from t where a<10 limit 1;") - require.EqualError(t, err, "[planner:1815]Internal : Can't find a proper physical plan for this query") - - // The partition table can not keep order - tk.MustExec("analyze table th;") - err = tk.ExecToErr("select a from th where a<1 order by a limit 1;") - require.NoError(t, err) - - err = tk.ExecToErr("select /*+ order_index(th, a) */ a from th where a<1 order by a limit 1;") - require.EqualError(t, err, "[planner:1815]Internal : Can't find a proper physical plan for this query") - - var input []string - var output []struct { - SQL string - Plan []string - Warn []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(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} - -func TestKeepOrderHintWithBinding(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") - tk.MustExec("create table t1(a int, b int, index idx_a(a));") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1(a int, b int, index idx_a(a));") // create binding for order_index hint tk.MustExec("select * from t1 where a<10 order by a limit 1;") @@ -1434,80 +502,6 @@ func TestKeepOrderHintWithBinding(t *testing.T) { require.Equal(t, len(res), 0) } -func TestViewHint(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 view if exists v, v1, v2") - tk.MustExec("drop table if exists t, t1, t2") - tk.MustExec("create table t(a int, b int);") - tk.MustExec("create table t1(a int, b int);") - tk.MustExec("create table t2(a int, b int);") - tk.MustExec("create definer='root'@'localhost' view v as select t.a, t.b from t join (select count(*) as a from t1 join t2 on t1.b=t2.b group by t2.a) tt on t.a = tt.a;") - tk.MustExec("create definer='root'@'localhost' view v1 as select t.a, t.b from t join (select count(*) as a from t1 join v on t1.b=v.b group by v.a) tt on t.a = tt.a;") - tk.MustExec("create definer='root'@'localhost' view v2 as select t.a, t.b from t join (select count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") - - var input []string - var output []struct { - SQL string - Plan []string - Warn []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(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} - -func TestViewHintScope(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 view if exists v, v1, v2, v3, v4") - tk.MustExec("drop table if exists t, t1, t2, t3, t4") - tk.MustExec("create table t(a int, b int);") - 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)") - tk.MustExec("create table t4(a int, b int, index idx_a(a), index idx_b(b))") - tk.MustExec("create definer='root'@'localhost' view v as select t.a, t.b from t join (select count(*) as a from t1 join t2 join t3 where t1.b=t2.b and t2.a = t3.a group by t2.a) tt on t.a = tt.a;") - tk.MustExec("create definer='root'@'localhost' view v1 as select t.a, t.b from t join (select count(*) as a from t1 join v on t1.b=v.b group by v.a) tt on t.a = tt.a;") - tk.MustExec("create definer='root'@'localhost' view v2 as select t.a, t.b from t join (select count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") - tk.MustExec("create definer='root'@'localhost' view v3 as select /*+ merge_join(t) */ t.a, t.b from t join (select /*+ stream_agg() */ count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") - tk.MustExec("create definer='root'@'localhost' view v4 as select * from t4 where a > 2 and b > 3;") - - var input []string - var output []struct { - SQL string - Plan []string - Warn []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(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} - func TestViewHintWithBinding(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -1522,287 +516,22 @@ func TestViewHintWithBinding(t *testing.T) { tk.MustExec("create table t3(a int, b int)") tk.MustExec("create definer='root'@'localhost' view v as select t.a, t.b from t join (select count(*) as a from t1 join t2 join t3 where t1.b=t2.b and t2.a = t3.a group by t2.a) tt on t.a = tt.a;") tk.MustExec("create definer='root'@'localhost' view v1 as select t.a, t.b from t join (select count(*) as a from t1 join v on t1.b=v.b group by v.a) tt on t.a = tt.a;") - tk.MustExec("create definer='root'@'localhost' view v2 as select t.a, t.b from t join (select count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") - - tk.MustExec("select * from v2") - tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) - tk.MustExec("create global binding for select * from v2 using select /*+ qb_name(qb_v_2, v2.v1@sel_2 .v@sel_2 .@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v2. v1@sel_2 .v@sel_2 .@sel_1), merge_join(t@qb_v_1) */ * from v2;") - tk.MustExec("select * from v2") - tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) - res := tk.MustQuery("show global bindings").Rows() - require.Equal(t, res[0][0], "select * from `test` . `v2`") - require.Equal(t, res[0][1], "SELECT /*+ qb_name(`qb_v_2` , `v2`. `v1`@`sel_2`. `v`@`sel_2`. ``@`sel_2`) merge_join(`t1`@`qb_v_2`) stream_agg(@`qb_v_2`) qb_name(`qb_v_1` , `v2`. `v1`@`sel_2`. `v`@`sel_2`. ``@`sel_1`) merge_join(`t`@`qb_v_1`)*/ * FROM `test`.`v2`") - - tk.MustExec("drop global binding for select * from v2") - tk.MustExec("select * from v2") - tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) - res = tk.MustQuery("show global bindings").Rows() - require.Equal(t, len(res), 0) -} - -func TestAllViewHintType(t *testing.T) { - store := testkit.CreateMockStore(t, withMockTiFlash(2)) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("set @@session.tidb_allow_mpp=ON") - tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash, tikv'") - tk.MustExec("drop view if exists v, v1, v2, v3, v4, v5, v6, v7, v8, v9, v10, v11, v12") - tk.MustExec("drop table if exists t, t1, t2, t4, t3, t5") - tk.MustExec("create table t(a int not null, b int, index idx_a(a));") - tk.MustExec("create table t1(a int not null, b int, index idx_a(a));") - tk.MustExec("create table t2(a int, b int, index idx_a(a));") - tk.MustExec("create table t3(a int, b int, index idx_a(a));") - tk.MustExec("create table t4(a int, b int, index idx_a(a));") - tk.MustExec("create table t5(a int, b int, index idx_a(a), index idx_b(b));") - - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("create definer='root'@'localhost' view v as select t.a, t.b from t join t1 on t.a = t1.a;") - tk.MustExec("create definer='root'@'localhost' view v1 as select t2.a, t2.b from t2 join t3 join v where t2.b = t3.b and t3.a = v.a;") - tk.MustExec("create definer='root'@'localhost' view v2 as select t.a, t.b from t join (select count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") - tk.MustExec("create definer='root'@'localhost' view v3 as select * from t5 where a > 1 and b < 2;") - tk.MustExec("create definer='root'@'localhost' view v4 as select * from t5 where a > 1 or b < 2;") - tk.MustExec("create definer='root'@'localhost' view v5 as SELECT * FROM t WHERE EXISTS (SELECT 1 FROM t1 WHERE t1.b = t.b);") - tk.MustExec("create definer='root'@'localhost' view v6 as select * from t1 where t1.a < (select sum(t2.a) from t2 where t2.b = t1.b);") - tk.MustExec("create definer='root'@'localhost' view v7 as WITH CTE AS (SELECT * FROM t WHERE t.a < 60) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;") - tk.MustExec("create definer='root'@'localhost' view v8 as WITH CTE1 AS (SELECT b FROM t1), CTE2 AS (WITH CTE3 AS (SELECT a FROM t2), CTE4 AS (SELECT a FROM t3) SELECT CTE3.a FROM CTE3, CTE4) SELECT b FROM CTE1, CTE2 union select * from CTE1;") - tk.MustExec("create definer='root'@'localhost' view v9 as select sum(a) from t;") - tk.MustExec("create definer='root'@'localhost' view v10 as SELECT * FROM t WHERE a > 10 ORDER BY b LIMIT 1;") - tk.MustExec("create definer='root'@'localhost' view v11 as select a, sum(b) from t group by a") - tk.MustExec("create definer='root'@'localhost' view v12 as select t.a, t.b from t join t t1 on t.a = t1.a;") - - var input []string - var output []struct { - SQL string - Plan []string - Warn []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(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} - -func TestReadFromStorageHintAndIsolationRead(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, tt, ttt") - tk.MustExec("create table t(a int, b int, index ia(a))") - tk.MustExec("set @@session.tidb_isolation_read_engines=\"tikv\"") - - // 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, - } - } - - var input []string - var output []struct { - SQL string - Plan []string - Warn []string - } - integrationSuiteData := core.GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} - -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 := core.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 TestIsolationReadTiFlashUseIndexHint(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, index idx(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 - Plan []string - Warn []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(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} - -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 := 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(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 := core.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("create definer='root'@'localhost' view v2 as select t.a, t.b from t join (select count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") - 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)") + tk.MustExec("select * from v2") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) + tk.MustExec("create global binding for select * from v2 using select /*+ qb_name(qb_v_2, v2.v1@sel_2 .v@sel_2 .@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v2. v1@sel_2 .v@sel_2 .@sel_1), merge_join(t@qb_v_1) */ * from v2;") + tk.MustExec("select * from v2") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + res := tk.MustQuery("show global bindings").Rows() + require.Equal(t, res[0][0], "select * from `test` . `v2`") + require.Equal(t, res[0][1], "SELECT /*+ qb_name(`qb_v_2` , `v2`. `v1`@`sel_2`. `v`@`sel_2`. ``@`sel_2`) merge_join(`t1`@`qb_v_2`) stream_agg(@`qb_v_2`) qb_name(`qb_v_1` , `v2`. `v1`@`sel_2`. `v`@`sel_2`. ``@`sel_1`) merge_join(`t`@`qb_v_1`)*/ * FROM `test`.`v2`") - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } + tk.MustExec("drop global binding for select * from v2") + tk.MustExec("select * from v2") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) + res = tk.MustQuery("show global bindings").Rows() + require.Equal(t, len(res), 0) } func TestPartitionPruningWithDateType(t *testing.T) { @@ -1862,33 +591,6 @@ func TestErrNoDB(t *testing.T) { tk.MustExec("grant select on test1111 to test@'%'") } -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 := 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(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestINLJHintSmallTable(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -1902,84 +604,6 @@ func TestINLJHintSmallTable(t *testing.T) { tk.MustExec("explain format = 'brief' select /*+ TIDB_INLJ(t1) */ * from t1 join t2 on t1.a = t2.a") } -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 := 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(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 := 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(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestIndexMergeHint4CNF(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(id int primary key, a int, b int, c int, key(a), key(b), key(c))") - - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestInvisibleIndex(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2024,34 +648,6 @@ func TestInvisibleIndex(t *testing.T) { tk.MustExec("admin check index t i_a") } -// 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 := 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(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestTopNByConstFunc(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2061,73 +657,6 @@ func TestTopNByConstFunc(t *testing.T) { )) } -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 := 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(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestIndexHintWarning(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, key a(a), key b(b))") - tk.MustExec("create table t2(a int, b int, c int, key a(a), key b(b))") - var input []string - var output []struct { - SQL string - Warnings []string - } - integrationSuiteData := core.GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - tk.MustQuery(tt) - warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - output[i].Warnings = make([]string, len(warns)) - for j := range warns { - output[i].Warnings[j] = warns[j].Err.Error() - } - }) - tk.MustQuery(tt) - warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Len(t, warns, len(output[i].Warnings)) - for j := range warns { - require.Equal(t, stmtctx.WarnLevelWarning, warns[j].Level) - require.EqualError(t, warns[j].Err, output[i].Warnings[j]) - } - } - //Test view with index hint should result error - tk.MustExec("drop table if exists t1") - tk.MustExec("drop view if exists v1") - tk.MustExec("CREATE TABLE t1 (c1 INT PRIMARY KEY, c2 INT, INDEX (c2))") - tk.MustExec("INSERT INTO t1 VALUES (1,1), (2,2), (3,3)") - tk.MustExec("CREATE VIEW v1 AS SELECT c1, c2 FROM t1") - err := tk.ExecToErr("SELECT * FROM v1 USE INDEX (PRIMARY) WHERE c1=2") - require.True(t, terror.ErrorEqual(err, core.ErrKeyDoesNotExist)) -} - func TestIssue32672(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2176,88 +705,23 @@ func TestApproxCountDistinctInPartitionTable(t *testing.T) { " └─PartitionUnion 16000.00 root ", " ├─HashAgg 8000.00 root group by:test.t.b, funcs:approx_count_distinct(test.t.a)->Column#5, funcs:firstrow(test.t.b)->Column#6, funcs:firstrow(test.t.b)->test.t.b", " │ └─TableReader 10000.00 root data:TableFullScan", - " │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", - " └─HashAgg 8000.00 root group by:test.t.b, funcs:approx_count_distinct(test.t.a)->Column#5, funcs:firstrow(test.t.b)->Column#6, funcs:firstrow(test.t.b)->test.t.b", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo")) - tk.MustQuery("select approx_count_distinct(a), b from t group by b order by b desc").Check(testkit.Rows("1 2", "3 1")) -} - -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 := 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 " + 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 TestIssue17813(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists hash_partition_overflow") - tk.MustExec("create table hash_partition_overflow (c0 bigint unsigned) partition by hash(c0) partitions 3") - tk.MustExec("insert into hash_partition_overflow values (9223372036854775808)") - tk.MustQuery("select * from hash_partition_overflow where c0 = 9223372036854775808").Check(testkit.Rows("9223372036854775808")) - tk.MustQuery("select * from hash_partition_overflow where c0 in (1, 9223372036854775808)").Check(testkit.Rows("9223372036854775808")) -} - -func TestHintWithRequiredProperty(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("set @@session.tidb_executor_concurrency = 4;") - tk.MustExec("set @@session.tidb_hash_join_concurrency = 5;") - tk.MustExec("set @@session.tidb_distsql_scan_concurrency = 15;") - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int primary key, b int, c int, key b(b))") - var input []string - var output []struct { - SQL string - Plan []string - Warnings []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(tt).Rows()) - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - output[i].Warnings = make([]string, len(warnings)) - for j, warning := range warnings { - output[i].Warnings[j] = warning.Err.Error() - } - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Len(t, warnings, len(output[i].Warnings)) - for j, warning := range warnings { - require.EqualError(t, warning.Err, output[i].Warnings[j]) - } - } + " │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + " └─HashAgg 8000.00 root group by:test.t.b, funcs:approx_count_distinct(test.t.a)->Column#5, funcs:firstrow(test.t.b)->Column#6, funcs:firstrow(test.t.b)->test.t.b", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo")) + tk.MustQuery("select approx_count_distinct(a), b from t group by b order by b desc").Check(testkit.Rows("1 2", "3 1")) +} + +func TestIssue17813(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists hash_partition_overflow") + tk.MustExec("create table hash_partition_overflow (c0 bigint unsigned) partition by hash(c0) partitions 3") + tk.MustExec("insert into hash_partition_overflow values (9223372036854775808)") + tk.MustQuery("select * from hash_partition_overflow where c0 = 9223372036854775808").Check(testkit.Rows("9223372036854775808")) + tk.MustQuery("select * from hash_partition_overflow where c0 in (1, 9223372036854775808)").Check(testkit.Rows("9223372036854775808")) } func TestIssue15813(t *testing.T) { @@ -2305,40 +769,6 @@ func TestFullGroupByOrderBy(t *testing.T) { require.True(t, terror.ErrorEqual(err, core.ErrFieldNotInGroupBy)) } -func TestHintWithoutTableWarning(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, key a(a))") - tk.MustExec("create table t2(a int, b int, c int, key a(a))") - var input []string - var output []struct { - SQL string - Warnings []string - } - integrationSuiteData := core.GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - tk.MustQuery(tt) - warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - output[i].Warnings = make([]string, len(warns)) - for j := range warns { - output[i].Warnings[j] = warns[j].Err.Error() - } - }) - tk.MustQuery(tt) - warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Len(t, warns, len(output[i].Warnings)) - for j := range warns { - require.Equal(t, stmtctx.WarnLevelWarning, warns[j].Level) - require.EqualError(t, warns[j].Err, output[i].Warnings[j]) - } - } -} - func TestIssue15858(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2426,34 +856,6 @@ func TestTableDualWithRequiredProperty(t *testing.T) { tk.MustExec("select /*+ MERGE_JOIN(t1, t2) */ * from t1 partition (p0), t2 where t1.a > 100 and t1.a = t2.a") } -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 := 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(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestIssue16837(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2471,35 +873,6 @@ func TestIssue16837(t *testing.T) { tk.MustQuery("select /*+ use_index_merge(t,c,idx_ab) */ * from t where a = 1 or (e = 1 and c = 1)").Check(testkit.Rows()) } -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 := 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(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 TestIndexMergePartialScansClusteredIndex(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2712,88 +1085,6 @@ func TestIssue16407(t *testing.T) { tk.MustQuery("select /*+ use_index_merge(t) */ * from t where a=10 or b='x'").Check(testkit.Rows()) } -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 := 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].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 TestOptimizeHintOnPartitionTable(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 varchar(20), - primary key(a), key(b), key(c) - ) partition by range columns(a) ( - partition p0 values less than(6), - partition p1 values less than(11), - partition p2 values less than(16));`) - tk.MustExec(`insert into t values (1,1,"1"), (2,2,"2"), (8,8,"8"), (11,11,"11"), (15,15,"15")`) - tk.MustExec("set @@tidb_enable_index_merge = off") - defer func() { - tk.MustExec("set @@tidb_enable_index_merge = 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) - - var input []string - var output []struct { - SQL string - Plan []string - Warn []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].Warn = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warn...)) - } -} - func TestNotReadOnlySQLOnTiFlash(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2927,35 +1218,6 @@ func TestIssue16935(t *testing.T) { tk.MustQuery("SELECT * FROM t0 LEFT JOIN v0 ON TRUE WHERE v0.c0 IS NULL;") } -func TestAccessPathOnClusterIndex(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("drop table if exists t1") - tk.MustExec("create table t1 (a int, b varchar(20), c decimal(40,10), d int, primary key(a,b), key(c))") - tk.MustExec(`insert into t1 values (1,"111",1.1,11), (2,"222",2.2,12), (3,"333",3.3,13)`) - tk.MustExec("analyze table t1") - - var input []string - var output []struct { - SQL string - Plan []string - Res []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].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...)) - } -} - func TestClusterIndexUniqueDoubleRead(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2970,36 +1232,6 @@ func TestClusterIndexUniqueDoubleRead(t *testing.T) { tk.MustQuery("select * from t use index (uuk);").Check(testkit.Rows("a a1 1 11", "b b1 2 22", "c c1 3 33")) } -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 := 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].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 TestIssue18984(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -3757,35 +1989,6 @@ func TestExplainAnalyzeDML2(t *testing.T) { } } -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 := 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 " + tt).Rows()) - }) - tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestPartialBatchPointGet(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -3964,34 +2167,6 @@ func TestIssue14481(t *testing.T) { tk.MustExec("drop table t") } -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 := 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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - func TestQueryBlockTableAliasInHint(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/planner/core/internal/BUILD.bazel b/planner/core/internal/BUILD.bazel index 8d0fefac9394a..3c4c45696c658 100644 --- a/planner/core/internal/BUILD.bazel +++ b/planner/core/internal/BUILD.bazel @@ -10,6 +10,10 @@ go_library( "//expression/aggregation", "//parser/model", "//sessionctx", + "//store/mockstore", + "//store/mockstore/unistore", + "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//testutils", ], ) diff --git a/planner/core/internal/testkit.go b/planner/core/internal/testkit.go index 6c3e7f6b022f1..214fc89b01b04 100644 --- a/planner/core/internal/testkit.go +++ b/planner/core/internal/testkit.go @@ -15,13 +15,18 @@ package internal import ( + "fmt" "testing" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/testutils" ) // SetTiFlashReplica is to set TiFlash replica @@ -49,3 +54,23 @@ func WrapCastForAggFuncs(sctx sessionctx.Context, aggFuncs []*aggregation.AggFun } } } + +// WithMockTiFlash sets the mockStore to have N TiFlash stores (naming as tiflash0, tiflash1, ...). +func WithMockTiFlash(nodes int) mockstore.MockTiKVStoreOption { + return mockstore.WithMultipleOptions( + mockstore.WithClusterInspector(func(c testutils.Cluster) { + mockCluster := c.(*unistore.Cluster) + _, _, region1 := mockstore.BootstrapWithSingleStore(c) + tiflashIdx := 0 + for tiflashIdx < nodes { + store2 := c.AllocID() + peer2 := c.AllocID() + addr2 := fmt.Sprintf("tiflash%d", tiflashIdx) + mockCluster.AddStore(store2, addr2, &metapb.StoreLabel{Key: "engine", Value: "tiflash"}) + mockCluster.AddPeer(region1, store2, peer2) + tiflashIdx++ + } + }), + mockstore.WithStoreType(mockstore.EmbedUnistore), + ) +} diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index a8a7a1918cfc0..f87e0a49f7173 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -18,14 +18,9 @@ import ( "context" "fmt" "math" - "strings" "testing" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser" @@ -34,53 +29,15 @@ import ( "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/planner" "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/planner/core/internal" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/external" - "github.com/pingcap/tidb/testkit/testdata" - "github.com/pingcap/tidb/util/hint" "github.com/stretchr/testify/require" - "github.com/tikv/client-go/v2/testutils" ) -func TestDAGPlanBuilderSimpleCase(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") - var input []string - var output []struct { - SQL string - Best string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - for i, tt := range input { - comment := fmt.Sprintf("case: %v, sql: %s", i, tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - require.NoError(t, sessiontxn.NewTxn(context.Background(), tk.Session())) - p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Best = core.ToString(p) - }) - require.Equal(t, output[i].Best, core.ToString(p), comment) - } -} - func TestAnalyzeBuildSucc(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -181,395 +138,6 @@ func TestAnalyzeSetRate(t *testing.T) { } } -func TestDAGPlanBuilderJoin(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - sessionVars := tk.Session().GetSessionVars() - sessionVars.ExecutorConcurrency = 4 - sessionVars.SetDistSQLScanConcurrency(15) - sessionVars.SetHashJoinConcurrency(5) - - var input []string - var output []struct { - SQL string - Best string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - for i, tt := range input { - comment := fmt.Sprintf("case:%v sql:%s", i, tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - - p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Best = core.ToString(p) - }) - require.Equal(t, output[i].Best, core.ToString(p), comment) - } -} - -func TestDAGPlanBuilderSubquery(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 sql_mode='STRICT_TRANS_TABLES'") // disable only full group by - sessionVars := tk.Session().GetSessionVars() - sessionVars.SetHashAggFinalConcurrency(1) - sessionVars.SetHashAggPartialConcurrency(1) - sessionVars.SetHashJoinConcurrency(5) - sessionVars.SetDistSQLScanConcurrency(15) - sessionVars.ExecutorConcurrency = 4 - var input []string - var output []struct { - SQL string - Best string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - for i, tt := range input { - comment := fmt.Sprintf("input: %s", tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - - p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Best = core.ToString(p) - }) - require.Equal(t, output[i].Best, core.ToString(p), fmt.Sprintf("input: %s", tt)) - } -} - -func TestDAGPlanTopN(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - var input []string - var output []struct { - SQL string - Best string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - for i, tt := range input { - comment := fmt.Sprintf("case:%v sql:%s", i, tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - - p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Best = core.ToString(p) - }) - require.Equal(t, output[i].Best, core.ToString(p), comment) - } -} - -func assertSameHints(t *testing.T, expected, actual []*ast.TableOptimizerHint) { - expectedStr := make([]string, 0, len(expected)) - actualStr := make([]string, 0, len(actual)) - for _, h := range expected { - expectedStr = append(expectedStr, hint.RestoreTableOptimizerHint(h)) - } - for _, h := range actual { - actualStr = append(actualStr, hint.RestoreTableOptimizerHint(h)) - } - require.ElementsMatch(t, expectedStr, actualStr) -} - -func TestDAGPlanBuilderBasePhysicalPlan(t *testing.T) { - store := testkit.CreateMockStore(t) - - se, err := session.CreateSession4Test(store) - require.NoError(t, err) - _, err = se.Execute(context.Background(), "use test") - require.NoError(t, err) - - var input []string - var output []struct { - SQL string - Best string - Hints string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - for i, tt := range input { - comment := fmt.Sprintf("input: %s", tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - - err = core.Preprocess(context.Background(), se, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: is})) - require.NoError(t, err) - p, _, err := planner.Optimize(context.TODO(), se, stmt, is) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Best = core.ToString(p) - output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) - }) - require.Equal(t, output[i].Best, core.ToString(p), fmt.Sprintf("input: %s", tt)) - hints := core.GenHintsFromPhysicalPlan(p) - - // test the new genHints code - flat := core.FlattenPhysicalPlan(p, false) - newHints := core.GenHintsFromFlatPlan(flat) - assertSameHints(t, hints, newHints) - - require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(hints), fmt.Sprintf("input: %s", tt)) - } -} - -func TestDAGPlanBuilderUnion(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - var input []string - var output []struct { - SQL string - Best string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - for i, tt := range input { - comment := fmt.Sprintf("case:%v sql:%s", i, tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - - p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Best = core.ToString(p) - }) - require.Equal(t, output[i].Best, core.ToString(p), comment) - } -} - -func TestDAGPlanBuilderUnionScan(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)") - - var input []string - var output []struct { - SQL string - Best string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - - p := parser.New() - for i, tt := range input { - tk.MustExec("begin;") - tk.MustExec("insert into t values(2, 2, 2);") - - comment := fmt.Sprintf("input: %s", tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - dom := domain.GetDomain(tk.Session()) - require.NoError(t, dom.Reload()) - plan, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, dom.InfoSchema()) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Best = core.ToString(plan) - }) - require.Equal(t, output[i].Best, core.ToString(plan), fmt.Sprintf("input: %s", tt)) - tk.MustExec("rollback;") - } -} - -func TestDAGPlanBuilderAgg(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 sql_mode='STRICT_TRANS_TABLES'") // disable only full group by - sessionVars := tk.Session().GetSessionVars() - sessionVars.SetHashAggFinalConcurrency(1) - sessionVars.SetHashAggPartialConcurrency(1) - sessionVars.SetDistSQLScanConcurrency(15) - sessionVars.ExecutorConcurrency = 4 - - var input []string - var output []struct { - SQL string - Best string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - for i, tt := range input { - comment := fmt.Sprintf("input: %s", tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - - p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Best = core.ToString(p) - }) - require.Equal(t, output[i].Best, core.ToString(p), fmt.Sprintf("input: %s", tt)) - } -} - -func TestRefine(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - - var input []string - var output []struct { - SQL string - Best string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - for i, tt := range input { - comment := fmt.Sprintf("input: %s", tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - sc := tk.Session().GetSessionVars().StmtCtx - sc.IgnoreTruncate = false - p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) - require.NoError(t, err, comment) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Best = core.ToString(p) - }) - require.Equal(t, output[i].Best, core.ToString(p), comment) - } -} - -func TestAggEliminator(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("set sql_mode='STRICT_TRANS_TABLES'") // disable only full group by - var input []string - var output []struct { - SQL string - Best string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - for i, tt := range input { - comment := fmt.Sprintf("input: %s", tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - sc := tk.Session().GetSessionVars().StmtCtx - sc.IgnoreTruncate = false - p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Best = core.ToString(p) - }) - require.Equal(t, output[i].Best, core.ToString(p), fmt.Sprintf("input: %s", tt)) - } -} - -func TestINMJHint(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - 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 primary key, b int not null)") - tk.MustExec("create table t2(a int primary key, b int not null)") - tk.MustExec("insert into t1 values(1,1),(2,2)") - tk.MustExec("insert into t2 values(1,1),(2,1)") - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) - } -} - -func TestEliminateMaxOneRow(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1;") - tk.MustExec("drop table if exists t2;") - tk.MustExec("drop table if exists t3;") - tk.MustExec("create table t1(a int(11) DEFAULT NULL, b int(11) DEFAULT NULL, UNIQUE KEY idx_a (a))") - tk.MustExec("create table t2(a int(11) DEFAULT NULL, b int(11) DEFAULT NULL)") - tk.MustExec("create table t3(a int(11) DEFAULT NULL, b int(11) DEFAULT NULL, c int(11) DEFAULT NULL, UNIQUE KEY idx_abc (a, b, c))") - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(ts).Check(testkit.Rows(output[i].Result...)) - } -} - type overrideStore struct{ kv.Storage } func (store overrideStore) GetClient() kv.Client { @@ -601,81 +169,6 @@ func TestRequestTypeSupportedOff(t *testing.T) { require.Equal(t, expect, core.ToString(p), fmt.Sprintf("sql: %s", sql)) } -func TestIndexJoinUnionScan(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("create table t (a int primary key, b int, index idx(a))") - tk.MustExec("create table tt (a int primary key) partition by range (a) (partition p0 values less than (100), partition p1 values less than (200))") - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) - - var input [][]string - var output []struct { - SQL []string - Plan []string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - - for i, ts := range input { - tk.MustExec("begin") - for j, tt := range ts { - if j != len(ts)-1 { - tk.MustExec(tt) - } - testdata.OnRecord(func() { - output[i].SQL = ts - if j == len(ts)-1 { - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - } - }) - if j == len(ts)-1 { - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } - } - tk.MustExec("rollback") - } -} - -func TestMergeJoinUnionScan(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t1 (c_int int, c_str varchar(40), primary key (c_int))") - tk.MustExec("create table t2 (c_int int, c_str varchar(40), primary key (c_int))") - tk.MustExec("insert into t1 (`c_int`, `c_str`) values (11, 'keen williamson'), (10, 'gracious hermann')") - tk.MustExec("insert into t2 (`c_int`, `c_str`) values (10, 'gracious hermann')") - - var input [][]string - var output []struct { - SQL []string - Plan []string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - - for i, ts := range input { - tk.MustExec("begin") - for j, tt := range ts { - if j != len(ts)-1 { - tk.MustExec(tt) - } - testdata.OnRecord(func() { - output[i].SQL = ts - if j == len(ts)-1 { - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - } - }) - if j == len(ts)-1 { - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } - } - tk.MustExec("rollback") - } -} - func TestDoSubQuery(t *testing.T) { store := testkit.CreateMockStore(t) @@ -724,128 +217,8 @@ func TestIndexLookupCartesianJoin(t *testing.T) { require.True(t, terror.ErrorEqual(err, lastWarn.Err)) } -func TestSemiJoinToInner(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - - var input []string - var output []struct { - SQL string - Best string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - for i, tt := range input { - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err) - p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Best = core.ToString(p) - }) - require.Equal(t, output[i].Best, core.ToString(p)) - } -} - -func TestUnmatchedTableInHint(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - var input []string - var output []struct { - SQL string - Warning string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - for i, test := range input { - tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) - stmt, err := p.ParseOneStmt(test, "", "") - require.NoError(t, err) - _, _, err = planner.Optimize(context.TODO(), tk.Session(), stmt, is) - require.NoError(t, err) - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - testdata.OnRecord(func() { - output[i].SQL = test - if len(warnings) > 0 { - output[i].Warning = warnings[0].Err.Error() - } - }) - if output[i].Warning == "" { - require.Len(t, warnings, 0) - } else { - require.Len(t, warnings, 1) - require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) - require.Equal(t, output[i].Warning, warnings[0].Err.Error()) - } - } -} - -// withMockTiFlash sets the mockStore to have N TiFlash stores (naming as tiflash0, tiflash1, ...). -func withMockTiFlash(nodes int) mockstore.MockTiKVStoreOption { - return mockstore.WithMultipleOptions( - mockstore.WithClusterInspector(func(c testutils.Cluster) { - mockCluster := c.(*unistore.Cluster) - _, _, region1 := mockstore.BootstrapWithSingleStore(c) - tiflashIdx := 0 - for tiflashIdx < nodes { - store2 := c.AllocID() - peer2 := c.AllocID() - addr2 := fmt.Sprintf("tiflash%d", tiflashIdx) - mockCluster.AddStore(store2, addr2, &metapb.StoreLabel{Key: "engine", Value: "tiflash"}) - mockCluster.AddPeer(region1, store2, peer2) - tiflashIdx++ - } - }), - mockstore.WithStoreType(mockstore.EmbedUnistore), - ) -} - -func TestMPPHints(t *testing.T) { - store := testkit.CreateMockStore(t, withMockTiFlash(2)) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("create table t (a int, b int, c int, index idx_a(a), index idx_b(b))") - tk.MustExec("alter table t set tiflash replica 1") - tk.MustExec("set @@session.tidb_allow_mpp=ON") - tk.MustExec("create definer='root'@'localhost' view v as select a, sum(b) from t group by a, c;") - tk.MustExec("create definer='root'@'localhost' view v1 as select t1.a from t t1, t t2 where t1.a=t2.a;") - tb := external.GetTableByName(t, tk, "test", "t") - err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) - require.NoError(t, err) - - var input []string - var output []struct { - SQL string - Plan []string - Warn []string - } - - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } -} - func TestMPPHintsScope(t *testing.T) { - store := testkit.CreateMockStore(t, withMockTiFlash(2)) + store := testkit.CreateMockStore(t, internal.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_cost_model_version=2") @@ -884,7 +257,7 @@ func TestMPPHintsScope(t *testing.T) { } func TestMPPHintsWithBinding(t *testing.T) { - store := testkit.CreateMockStore(t, withMockTiFlash(2)) + store := testkit.CreateMockStore(t, internal.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_cost_model_version=2") @@ -932,201 +305,7 @@ func TestMPPHintsWithBinding(t *testing.T) { require.Equal(t, len(res), 0) } -func TestHintScope(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - - var input []string - var output []struct { - SQL string - Best string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - - for i, test := range input { - comment := fmt.Sprintf("case:%v sql:%s", i, test) - stmt, err := p.ParseOneStmt(test, "", "") - require.NoError(t, err, comment) - - p, _, err := planner.Optimize(context.Background(), tk.Session(), stmt, is) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = test - output[i].Best = core.ToString(p) - }) - require.Equal(t, output[i].Best, core.ToString(p)) - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Len(t, warnings, 0, comment) - } -} - -func TestJoinHints(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - - var input []string - var output []struct { - SQL string - Best string - Warning string - Hints string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - ctx := context.Background() - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - - for i, test := range input { - comment := fmt.Sprintf("case:%v sql:%s", i, test) - stmt, err := p.ParseOneStmt(test, "", "") - require.NoError(t, err, comment) - - tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) - p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) - require.NoError(t, err) - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - - testdata.OnRecord(func() { - output[i].SQL = test - output[i].Best = core.ToString(p) - if len(warnings) > 0 { - output[i].Warning = warnings[0].Err.Error() - } - output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) - }) - require.Equal(t, output[i].Best, core.ToString(p)) - if output[i].Warning == "" { - require.Len(t, warnings, 0) - } else { - require.Len(t, warnings, 1, fmt.Sprintf("%v", warnings)) - require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) - require.Equal(t, output[i].Warning, warnings[0].Err.Error()) - } - hints := core.GenHintsFromPhysicalPlan(p) - - // test the new genHints code - flat := core.FlattenPhysicalPlan(p, false) - newHints := core.GenHintsFromFlatPlan(flat) - assertSameHints(t, hints, newHints) - - require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(hints), comment) - } -} - -func TestAggregationHints(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - sessionVars := tk.Session().GetSessionVars() - sessionVars.SetHashAggFinalConcurrency(1) - sessionVars.SetHashAggPartialConcurrency(1) - - var input []struct { - SQL string - AggPushDown bool - } - var output []struct { - SQL string - Best string - Warning string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - ctx := context.Background() - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - for i, test := range input { - comment := fmt.Sprintf("case: %v sql: %v", i, test) - tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) - tk.Session().GetSessionVars().AllowAggPushDown = test.AggPushDown - - stmt, err := p.ParseOneStmt(test.SQL, "", "") - require.NoError(t, err, comment) - - p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) - require.NoError(t, err) - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - - testdata.OnRecord(func() { - output[i].SQL = test.SQL - output[i].Best = core.ToString(p) - if len(warnings) > 0 { - output[i].Warning = warnings[0].Err.Error() - } - }) - require.Equal(t, output[i].Best, core.ToString(p), comment) - if output[i].Warning == "" { - require.Len(t, warnings, 0) - } else { - require.Len(t, warnings, 1, fmt.Sprintf("%v", warnings)) - require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) - require.Equal(t, output[i].Warning, warnings[0].Err.Error()) - } - } -} - -func TestSemiJoinRewriteHints(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("create table t(a int, b int, c int)") - - sessionVars := tk.Session().GetSessionVars() - sessionVars.SetHashAggFinalConcurrency(1) - sessionVars.SetHashAggPartialConcurrency(1) - - var input []string - var output []struct { - SQL string - Plan []string - Warning string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - ctx := context.Background() - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - for i, test := range input { - comment := fmt.Sprintf("case: %v sql: %v", i, test) - tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) - - stmt, err := p.ParseOneStmt(test, "", "") - require.NoError(t, err, comment) - - _, _, err = planner.Optimize(ctx, tk.Session(), stmt, is) - require.NoError(t, err) - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - - testdata.OnRecord(func() { - output[i].SQL = test - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief'" + test).Rows()) - if len(warnings) > 0 { - output[i].Warning = warnings[0].Err.Error() - } - }) - tk.MustQuery("explain format = 'brief'" + test).Check(testkit.Rows(output[i].Plan...)) - if output[i].Warning == "" { - require.Len(t, warnings, 0) - } else { - require.Len(t, warnings, 1, fmt.Sprintf("%v", warnings)) - require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) - require.Equal(t, output[i].Warning, warnings[0].Err.Error()) - } - } -} - -func TestExplainJoinHints(t *testing.T) { +func TestExplainJoinHints(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -1140,425 +319,6 @@ func TestExplainJoinHints(t *testing.T) { )) } -func TestAggToCopHint(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists ta") - tk.MustExec("create table ta(a int, b int, index(a))") - - var ( - input []string - output []struct { - SQL string - Best string - Warning string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - - ctx := context.Background() - is := domain.GetDomain(tk.Session()).InfoSchema() - p := parser.New() - for i, test := range input { - comment := fmt.Sprintf("case:%v sql:%s", i, test) - testdata.OnRecord(func() { - output[i].SQL = test - }) - require.Equal(t, output[i].SQL, test, comment) - - tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) - - stmt, err := p.ParseOneStmt(test, "", "") - require.NoError(t, err, comment) - - p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) - require.NoError(t, err, comment) - planString := core.ToString(p) - testdata.OnRecord(func() { - output[i].Best = planString - }) - require.Equal(t, output[i].Best, planString, comment) - - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - testdata.OnRecord(func() { - if len(warnings) > 0 { - output[i].Warning = warnings[0].Err.Error() - } - }) - if output[i].Warning == "" { - require.Len(t, warnings, 0) - } else { - require.Len(t, warnings, 1, fmt.Sprintf("%v", warnings)) - require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) - require.Equal(t, output[i].Warning, warnings[0].Err.Error()) - } - } -} - -func TestLimitToCopHint(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 tn") - tk.MustExec("create table tn(a int, b int, c int, d int, key (a, b, c, d))") - tk.MustExec(`set tidb_opt_limit_push_down_threshold=0`) - - var ( - input []string - output []struct { - SQL string - Plan []string - Warning []string - } - ) - - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - - comment := fmt.Sprintf("case:%v sql:%s", i, ts) - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - testdata.OnRecord(func() { - if len(warnings) > 0 { - output[i].Warning = make([]string, len(warnings)) - for j, warning := range warnings { - output[i].Warning[j] = warning.Err.Error() - } - } - }) - if len(output[i].Warning) == 0 { - require.Len(t, warnings, 0) - } else { - require.Len(t, warnings, len(output[i].Warning), comment) - for j, warning := range warnings { - require.Equal(t, stmtctx.WarnLevelWarning, warning.Level, comment) - require.Equal(t, output[i].Warning[j], warning.Err.Error(), comment) - } - } - } -} - -func TestCTEMergeHint(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists tc") - tk.MustExec("drop table if exists te") - tk.MustExec("drop table if exists t1") - tk.MustExec("drop table if exists t2") - tk.MustExec("drop table if exists t3") - tk.MustExec("drop table if exists t4") - tk.MustExec("drop view if exists v") - tk.MustExec("create table tc(a int)") - tk.MustExec("create table te(c int)") - tk.MustExec("create table t1(a int)") - tk.MustExec("create table t2(b int)") - tk.MustExec("create table t3(c int)") - tk.MustExec("create table t4(d int)") - tk.MustExec("insert into tc values (1), (5), (10), (15), (20), (30), (50);") - tk.MustExec("insert into te values (1), (5), (10), (25), (40), (60), (100);") - tk.MustExec("insert into t1 values (1), (5), (10), (25), (40), (60), (100);") - tk.MustExec("insert into t2 values (1), (5), (10), (25), (40), (60), (100);") - tk.MustExec("insert into t3 values (1), (5), (10), (25), (40), (60), (100);") - tk.MustExec("insert into t4 values (1), (5), (10), (25), (40), (60), (100);") - tk.MustExec("analyze table tc;") - tk.MustExec("analyze table te;") - tk.MustExec("analyze table t1;") - tk.MustExec("analyze table t2;") - tk.MustExec("analyze table t3;") - tk.MustExec("analyze table t4;") - tk.MustExec("create definer='root'@'localhost' view v as select * from tc") - var ( - input []string - output []struct { - SQL string - Plan []string - Warning []string - } - ) - - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - - comment := fmt.Sprintf("case:%v sql:%s", i, ts) - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - testdata.OnRecord(func() { - if len(warnings) > 0 { - output[i].Warning = make([]string, len(warnings)) - for j, warning := range warnings { - output[i].Warning[j] = warning.Err.Error() - } - } - }) - if len(output[i].Warning) == 0 { - require.Len(t, warnings, 0) - } else { - require.Len(t, warnings, len(output[i].Warning), comment) - for j, warning := range warnings { - require.Equal(t, stmtctx.WarnLevelWarning, warning.Level, comment) - require.Equal(t, output[i].Warning[j], warning.Err.Error(), comment) - } - } - } -} - -func TestForceInlineCTE(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(11));") - tk.MustExec("insert into t values (1), (5), (10), (15), (20), (30), (50);") - - var ( - input []string - output []struct { - SQL string - Plan []string - Warning []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - }) - if strings.HasPrefix(ts, "set") { - tk.MustExec(ts) - continue - } - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + ts).Rows()) - }) - tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - - comment := fmt.Sprintf("case:%v sql:%s", i, ts) - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - testdata.OnRecord(func() { - if len(warnings) > 0 { - output[i].Warning = make([]string, len(warnings)) - for j, warning := range warnings { - output[i].Warning[j] = warning.Err.Error() - } - } - }) - if len(output[i].Warning) == 0 { - require.Len(t, warnings, 0) - } else { - require.Len(t, warnings, len(output[i].Warning), comment) - for j, warning := range warnings { - require.Equal(t, stmtctx.WarnLevelWarning, warning.Level, comment) - require.Equal(t, output[i].Warning[j], warning.Err.Error(), comment) - } - } - } -} - -func TestSingleConsumerCTE(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(11));") - tk.MustExec("insert into t values (1), (5), (10), (15), (20), (30), (50);") - - var ( - input []string - output []struct { - SQL string - Plan []string - Warning []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - }) - if strings.HasPrefix(ts, "set") { - tk.MustExec(ts) - continue - } - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + ts).Rows()) - }) - tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestPushdownDistinctEnable(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - vars := []string{ - fmt.Sprintf("set @@session.%s = 1", variable.TiDBOptDistinctAggPushDown), - "set session tidb_opt_agg_push_down = 1", - "set tidb_cost_model_version = 2", - } - doTestPushdownDistinct(t, vars, input, output) -} - -func TestPushdownDistinctDisable(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - } - ) - - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - vars := []string{ - fmt.Sprintf("set @@session.%s = 0", variable.TiDBOptDistinctAggPushDown), - "set session tidb_opt_agg_push_down = 1", - } - doTestPushdownDistinct(t, vars, input, output) -} - -func TestPushdownDistinctEnableAggPushDownDisable(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - vars := []string{ - fmt.Sprintf("set @@session.%s = 1", variable.TiDBOptDistinctAggPushDown), - "set session tidb_opt_agg_push_down = 0", - "set tidb_cost_model_version=2", - } - doTestPushdownDistinct(t, vars, input, output) -} - -func doTestPushdownDistinct(t *testing.T, vars, input []string, output []struct { - SQL string - Plan []string - Result []string -}) { - 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, index(c))") - tk.MustExec("insert into t values (1, 1, 1), (1, 1, 3), (1, 2, 3), (2, 1, 3), (1, 2, NULL);") - - tk.MustExec("drop table if exists pt") - tk.MustExec(`CREATE TABLE pt (a int, b int) PARTITION BY RANGE (a) ( - PARTITION p0 VALUES LESS THAN (2), - PARTITION p1 VALUES LESS THAN (100) - );`) - - tk.MustExec("drop table if exists tc;") - tk.MustExec("CREATE TABLE `tc`(`timestamp` timestamp NULL DEFAULT NULL, KEY `idx_timestamp` (`timestamp`)) PARTITION BY RANGE ( UNIX_TIMESTAMP(`timestamp`) ) (PARTITION `p2020072312` VALUES LESS THAN (1595480400),PARTITION `p2020072313` VALUES LESS THAN (1595484000));") - - tk.MustExec("drop table if exists ta") - tk.MustExec("create table ta(a int);") - tk.MustExec("insert into ta values(1), (1);") - tk.MustExec("drop table if exists tb") - tk.MustExec("create table tb(a int);") - tk.MustExec("insert into tb values(1), (1);") - - tk.MustExec("set session sql_mode=''") - tk.MustExec(fmt.Sprintf("set session %s=1", variable.TiDBHashAggPartialConcurrency)) - tk.MustExec(fmt.Sprintf("set session %s=1", variable.TiDBHashAggFinalConcurrency)) - - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) - - for _, v := range vars { - tk.MustExec(v) - } - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) - } -} - -func TestGroupConcatOrderby(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - 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, name int)") - tk.MustExec("insert into test values(1, 10);") - tk.MustExec("insert into test values(1, 20);") - tk.MustExec("insert into test values(1, 30);") - tk.MustExec("insert into test values(2, 20);") - tk.MustExec("insert into test values(3, 200);") - tk.MustExec("insert into test values(3, 500);") - - tk.MustExec("drop table if exists ptest;") - tk.MustExec("CREATE TABLE ptest (id int,name int) PARTITION BY RANGE ( id ) " + - "(PARTITION `p0` VALUES LESS THAN (2), PARTITION `p1` VALUES LESS THAN (11))") - tk.MustExec("insert into ptest select * from test;") - tk.MustExec(fmt.Sprintf("set session tidb_opt_distinct_agg_push_down = %v", 1)) - tk.MustExec(fmt.Sprintf("set session tidb_opt_agg_push_down = %v", 1)) - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(ts).Check(testkit.Rows(output[i].Result...)) - } -} - func TestHintAlias(t *testing.T) { store := testkit.CreateMockStore(t) @@ -1602,201 +362,6 @@ func TestHintAlias(t *testing.T) { } } -func TestIndexHint(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - var input []string - var output []struct { - SQL string - Best string - HasWarn bool - Hints string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - ctx := context.Background() - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - - for i, test := range input { - comment := fmt.Sprintf("case:%v sql:%s", i, test) - tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) - - stmt, err := p.ParseOneStmt(test, "", "") - require.NoError(t, err, comment) - - p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = test - output[i].Best = core.ToString(p) - output[i].HasWarn = len(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) > 0 - output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) - }) - require.Equal(t, output[i].Best, core.ToString(p), comment) - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - if output[i].HasWarn { - require.Len(t, warnings, 1, comment) - } else { - require.Len(t, warnings, 0, comment) - } - hints := core.GenHintsFromPhysicalPlan(p) - - // test the new genHints code - flat := core.FlattenPhysicalPlan(p, false) - newHints := core.GenHintsFromFlatPlan(flat) - assertSameHints(t, hints, newHints) - - require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(hints), comment) - } -} - -func TestIndexMergeHint(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - - var input []string - var output []struct { - SQL string - Best string - HasWarn bool - Hints string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - ctx := context.Background() - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - - for i, test := range input { - comment := fmt.Sprintf("case:%v sql:%s", i, test) - tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) - stmt, err := p.ParseOneStmt(test, "", "") - require.NoError(t, err, comment) - sctx := tk.Session() - err = executor.ResetContextOfStmt(sctx, stmt) - require.NoError(t, err) - p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = test - output[i].Best = core.ToString(p) - output[i].HasWarn = len(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) > 0 - output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) - }) - require.Equal(t, output[i].Best, core.ToString(p), comment) - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - if output[i].HasWarn { - require.Len(t, warnings, 1, comment) - } else { - require.Len(t, warnings, 0, comment) - } - hints := core.GenHintsFromPhysicalPlan(p) - - // test the new genHints code - flat := core.FlattenPhysicalPlan(p, false) - newHints := core.GenHintsFromFlatPlan(flat) - assertSameHints(t, hints, newHints) - - require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(hints), comment) - } -} - -func TestQueryBlockHint(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - - var input []string - var output []struct { - SQL string - Plan string - Hints string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - ctx := context.TODO() - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - - for i, tt := range input { - comment := fmt.Sprintf("case:%v sql: %s", i, tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - - p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) - require.NoError(t, err, comment) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = core.ToString(p) - output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) - }) - require.Equal(t, output[i].Plan, core.ToString(p), comment) - hints := core.GenHintsFromPhysicalPlan(p) - - // test the new genHints code - flat := core.FlattenPhysicalPlan(p, false) - newHints := core.GenHintsFromFlatPlan(flat) - assertSameHints(t, hints, newHints) - - require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(hints), comment) - } -} - -func TestInlineProjection(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`drop table if exists test.t1, test.t2;`) - tk.MustExec(`create table test.t1(a bigint, b bigint, index idx_a(a), index idx_b(b));`) - tk.MustExec(`create table test.t2(a bigint, b bigint, index idx_a(a), index idx_b(b));`) - - var input []string - var output []struct { - SQL string - Plan string - Hints string - } - is := domain.GetDomain(tk.Session()).InfoSchema() - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - - ctx := context.Background() - p := parser.New() - - for i, tt := range input { - comment := fmt.Sprintf("case:%v sql: %s", i, tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - - p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) - require.NoError(t, err, comment) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = core.ToString(p) - output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) - }) - require.Equal(t, output[i].Plan, core.ToString(p), comment) - hints := core.GenHintsFromPhysicalPlan(p) - - // test the new genHints code - flat := core.FlattenPhysicalPlan(p, false) - newHints := core.GenHintsFromFlatPlan(flat) - assertSameHints(t, hints, newHints) - - require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(hints), comment) - } -} - func TestDAGPlanBuilderSplitAvg(t *testing.T) { store := testkit.CreateMockStore(t) @@ -1865,71 +430,6 @@ func testDAGPlanBuilderSplitAvg(t *testing.T, root core.PhysicalPlan) { } } -func TestIndexJoinHint(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 test.t1, test.t2, test.t;`) - tk.MustExec(`create table test.t1(a bigint, b bigint, index idx_a(a), index idx_b(b));`) - tk.MustExec(`create table test.t2(a bigint, b bigint, index idx_a(a), index idx_b(b));`) - tk.MustExec("CREATE TABLE `t` ( `a` bigint(20) NOT NULL, `b` tinyint(1) DEFAULT NULL, `c` datetime DEFAULT NULL, `d` int(10) unsigned DEFAULT NULL, `e` varchar(20) DEFAULT NULL, `f` double DEFAULT NULL, `g` decimal(30,5) DEFAULT NULL, `h` float DEFAULT NULL, `i` date DEFAULT NULL, `j` timestamp NULL DEFAULT NULL, PRIMARY KEY (`a`), UNIQUE KEY `b` (`b`), KEY `c` (`c`,`d`,`e`), KEY `f` (`f`), KEY `g` (`g`,`h`), KEY `g_2` (`g`), UNIQUE KEY `g_3` (`g`), KEY `i` (`i`) );") - - var input []string - var output []struct { - SQL string - Plan string - } - - is := domain.GetDomain(tk.Session()).InfoSchema() - p := parser.New() - ctx := context.Background() - - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - comment := fmt.Sprintf("case:%v sql: %s", i, tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) - require.NoError(t, err, comment) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = core.ToString(p) - }) - require.Equal(t, output[i].Plan, core.ToString(p), comment) - } -} - -func TestDAGPlanBuilderWindow(t *testing.T) { - var input []string - var output []struct { - SQL string - Best string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - vars := []string{ - "set @@session.tidb_window_concurrency = 1", - } - doTestDAGPlanBuilderWindow(t, vars, input, output) -} - -func TestDAGPlanBuilderWindowParallel(t *testing.T) { - var input []string - var output []struct { - SQL string - Best string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - vars := []string{ - "set @@session.tidb_window_concurrency = 4", - } - doTestDAGPlanBuilderWindow(t, vars, input, output) -} - func TestTopNPushDownEmpty(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -1939,192 +439,6 @@ func TestTopNPushDownEmpty(t *testing.T) { tk.MustQuery("select extract(day_hour from 'ziy') as res from t order by res limit 1").Check(testkit.Rows()) } -func doTestDAGPlanBuilderWindow(t *testing.T, vars, input []string, output []struct { - SQL string - Best string -}) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - for _, v := range vars { - tk.MustExec(v) - } - - p := parser.New() - is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - - for i, tt := range input { - comment := fmt.Sprintf("case:%v sql:%s", i, tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - - err = sessiontxn.NewTxn(context.Background(), tk.Session()) - require.NoError(t, err) - p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) - require.NoError(t, err) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Best = core.ToString(p) - }) - require.Equal(t, output[i].Best, core.ToString(p), comment) - } -} - -func TestNominalSort(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - var input []string - var output []struct { - SQL string - Plan []string - Result []string - } - tk.MustExec("create table t (a int, b int, index idx_a(a), index idx_b(b))") - tk.MustExec("insert into t values(1, 1)") - tk.MustExec("insert into t values(1, 2)") - tk.MustExec("insert into t values(2, 4)") - tk.MustExec("insert into t values(3, 5)") - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(ts).Check(testkit.Rows(output[i].Result...)) - } -} - -func TestHintFromDiffDatabase(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`drop table if exists test.t1`) - tk.MustExec(`create table test.t1(a bigint, index idx_a(a));`) - tk.MustExec(`create table test.t2(a bigint, index idx_a(a));`) - tk.MustExec("drop database if exists test2") - tk.MustExec("create database test2") - tk.MustExec("use test2") - - var input []string - var output []struct { - SQL string - Plan string - } - is := domain.GetDomain(tk.Session()).InfoSchema() - p := parser.New() - ctx := context.Background() - - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - comment := fmt.Sprintf("case:%v sql: %s", i, tt) - stmt, err := p.ParseOneStmt(tt, "", "") - require.NoError(t, err, comment) - p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) - require.NoError(t, err, comment) - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = core.ToString(p) - }) - require.Equal(t, output[i].Plan, core.ToString(p), comment) - } -} - -func TestNthPlanHintWithExplain(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`drop table if exists test.tt`) - tk.MustExec(`create table test.tt (a int,b int, index(a), index(b));`) - tk.MustExec("insert into tt values (1, 1), (2, 2), (3, 4)") - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) - - var input []string - var output []struct { - SQL string - Plan []string - } - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - } - - // This assertion makes sure a query with or without nth_plan() hint output exactly the same plan(including plan ID). - // The query below is the same as queries in the testdata except for nth_plan() hint. - // Currently, its output is the same as the second test case in the testdata, which is `output[1]`. If this doesn't - // hold in the future, you may need to modify this. - tk.MustQuery("explain format = 'brief' select * from test.tt where a=1 and b=1").Check(testkit.Rows(output[1].Plan...)) -} - -func TestEnumIndex(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - 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(e enum('c','b','a',''), index idx(e))") - tk.MustExec("insert ignore into t values(0),(1),(2),(3),(4);") - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) - }) - tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) - } -} - -func TestIssue27233(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("CREATE TABLE `PK_S_MULTI_31` (\n `COL1` tinyint(45) NOT NULL,\n `COL2` tinyint(45) NOT NULL,\n PRIMARY KEY (`COL1`,`COL2`) /*T![clustered_index] NONCLUSTERED */\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;") - tk.MustExec("insert into PK_S_MULTI_31 values(122,100),(124,-22),(124,34),(127,103);") - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) - }) - tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) - } -} - func TestPossibleProperties(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2139,57 +453,6 @@ func TestPossibleProperties(t *testing.T) { )) } -func TestSelectionPartialPushDown(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - 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 as (a+1) virtual)") - tk.MustExec("create table t2(a int, b int as (a+1) virtual, c int, key idx_a(a))") - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) - }) - tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestIssue28316(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - 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)") - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) - }) - tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestIssue30965(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2208,218 +471,6 @@ func TestIssue30965(t *testing.T) { " └─TableRowIDScan 10.00 cop[tikv] table:t30965 keep order:false, stats:pseudo")) } -func TestSkewDistinctAgg(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - 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(11), `b` int(11), `c` int(11), `d` date)") - tk.MustExec("insert into t (a,b,c,d) value(1,4,5,'2019-06-01')") - tk.MustExec("insert into t (a,b,c,d) value(2,null,1,'2019-07-01')") - tk.MustExec("insert into t (a,b,c,d) value(3,4,5,'2019-08-01')") - tk.MustExec("insert into t (a,b,c,d) value(3,6,2,'2019-09-01')") - tk.MustExec("insert into t (a,b,c,d) value(10,4,null,'2020-06-01')") - tk.MustExec("insert into t (a,b,c,d) value(20,null,1,'2020-07-01')") - tk.MustExec("insert into t (a,b,c,d) value(30,4,5,'2020-08-01')") - tk.MustExec("insert into t (a,b,c,d) value(30,6,5,'2020-09-01')") - tk.MustQuery("select date_format(d,'%Y') as df, sum(a), count(b), count(distinct c) " + - "from t group by date_format(d,'%Y') order by df;").Check( - testkit.Rows("2019 9 3 3", "2020 90 3 2")) - tk.MustExec("set @@tidb_opt_skew_distinct_agg=1") - tk.MustQuery("select date_format(d,'%Y') as df, sum(a), count(b), count(distinct c) " + - "from t group by date_format(d,'%Y') order by df;").Check( - testkit.Rows("2019 9 3 3", "2020 90 3 2")) - tk.MustQuery("select count(distinct b), sum(c) from t group by a order by 1,2;").Check( - testkit.Rows("0 1", "0 1", "1 ", "1 5", "2 7", "2 10")) - tk.MustQuery("select count(distinct b) from t group by date_format(d,'%Y') order by 1;").Check( - testkit.Rows("2", "2")) - tk.MustQuery("select count(a), count(distinct b), max(b) from t group by date_format(d,'%Y') order by 1,2,3;").Check( - testkit.Rows("4 2 6", "4 2 6")) - tk.MustQuery("select count(a), count(distinct b), max(b) from t group by date_format(d,'%Y'),c order by 1,2,3;").Check( - testkit.Rows("1 0 ", "1 0 ", "1 1 4", "1 1 6", "2 1 4", "2 2 6")) - tk.MustQuery("select avg(distinct b), count(a), sum(b) from t group by date_format(d,'%Y'),c order by 1,2,3;").Check( - testkit.Rows(" 1 ", " 1 ", "4.0000 1 4", "4.0000 2 8", "5.0000 2 10", "6.0000 1 6")) - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + ts).Rows()) - }) - tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestHJBuildAndProbeHint(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - Warning []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - 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(a int primary key, b int not null)") - tk.MustExec("create table t2(a int primary key, b int not null)") - tk.MustExec("create table t3(a int primary key, b int not null)") - tk.MustExec("insert into t1 values(1,1),(2,2)") - tk.MustExec("insert into t2 values(1,1),(2,1)") - tk.MustExec("insert into t3 values(1,1),(2,1)") - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) - output[i].Warning = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) - tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warning...)) - } -} - -func TestHJBuildAndProbeHint4StaticPartitionTable(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - Warning []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - 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(a int, b int) partition by hash(a) partitions 4`) - tk.MustExec(`create table t2(a int, b int) partition by hash(a) partitions 5`) - tk.MustExec(`create table t3(a int, b int) partition by hash(b) partitions 3`) - tk.MustExec("insert into t1 values(1,1),(2,2)") - tk.MustExec("insert into t2 values(1,1),(2,1)") - tk.MustExec("insert into t3 values(1,1),(2,1)") - tk.MustExec(`set @@tidb_partition_prune_mode="static"`) - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) - output[i].Warning = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) - } -} - -func TestHJBuildAndProbeHint4DynamicPartitionTable(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") - - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - Warning []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2, t3") - tk.MustExec(`create table t1(a int, b int) partition by hash(a) partitions 4`) - tk.MustExec(`create table t2(a int, b int) partition by hash(a) partitions 5`) - tk.MustExec(`create table t3(a int, b int) partition by hash(b) partitions 3`) - tk.MustExec("insert into t1 values(1,1),(2,2)") - tk.MustExec("insert into t2 values(1,1),(2,1)") - tk.MustExec("insert into t3 values(1,1),(2,1)") - tk.MustExec(`set @@tidb_partition_prune_mode="dynamic"`) - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) - output[i].Warning = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) - } -} - -func TestHJBuildAndProbeHint4TiFlash(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - Warning []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - 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(a int primary key, b int not null)") - tk.MustExec("create table t2(a int primary key, b int not null)") - tk.MustExec("create table t3(a int primary key, b int not null)") - tk.MustExec("insert into t1 values(1,1),(2,2)") - tk.MustExec("insert into t2 values(1,1),(2,1)") - tk.MustExec("insert into t3 values(1,1),(2,1)") - // 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 { - tableName := tblInfo.Name.L - if tableName == "t1" || tableName == "t2" || tableName == "t3" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Warning = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestHJBuildAndProbeHintWithBinding(t *testing.T) { store := testkit.CreateMockStore(t) @@ -2453,52 +504,6 @@ func TestHJBuildAndProbeHintWithBinding(t *testing.T) { require.Equal(t, len(res), 0) } -func TestMPPSinglePartitionType(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - store, dom := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists employee") - tk.MustExec("create table employee(empid int, deptid int, salary decimal(10,2))") - tk.MustExec("set tidb_enforce_mpp=0") - - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tblInfo := range db.Tables { - if tblInfo.Name.L == "employee" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - }) - if strings.HasPrefix(ts, "set") { - tk.MustExec(ts) - continue - } - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) - }) - tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestPhysicalPlanMemoryTrace(t *testing.T) { // PhysicalSort ls := core.PhysicalSort{} @@ -2512,190 +517,3 @@ func TestPhysicalPlanMemoryTrace(t *testing.T) { pp.MPPPartitionCols = append(pp.MPPPartitionCols, &property.MPPPartitionColumn{}) require.Greater(t, pp.MemoryUsage(), size) } - -func TestNoDecorrelateHint(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - Result []string - Warning []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - 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)") - tk.MustExec("create table t2(a int primary key, b int)") - tk.MustExec("create table t3(a int, b int)") - tk.MustExec("insert into t1 values(1,1),(2,2)") - tk.MustExec("insert into t2 values(1,1),(2,1)") - tk.MustExec("insert into t3 values(1,1),(2,1)") - - tk.MustExec("create table ta(id int, code int, name varchar(20), index idx_ta_id(id),index idx_ta_name(name), index idx_ta_code(code))") - tk.MustExec("create table tb(id int, code int, name varchar(20), index idx_tb_id(id),index idx_tb_name(name))") - tk.MustExec("create table tc(id int, code int, name varchar(20), index idx_tc_id(id),index idx_tc_name(name))") - tk.MustExec("create table td(id int, code int, name varchar(20), index idx_tc_id(id),index idx_tc_name(name))") - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) - output[i].Warning = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) - tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warning...)) - } -} - -func TestCountStarForTikv(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - Warning []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=1") - tk.MustExec("create table t (a int(11) not null, b varchar(10) not null, c date not null, d char(1) not null, e bigint not null, f datetime not null, g bool not null, h bool )") - tk.MustExec("create table t_pick_row_id (a char(20) not null)") - - // tikv - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestCountStarForTiFlash(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - Warning []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=1") - tk.MustExec("create table t (a int(11) not null, b varchar(10) not null, c date not null, d char(1) not null, e bigint not null, f datetime not null, g bool not null, h bool )") - tk.MustExec("create table t_pick_row_id (a char(20) not null)") - - // tiflash - dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tblInfo := range db.Tables { - tableName := tblInfo.Name.L - if tableName == "t" || tableName == "t_pick_row_id" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestHashAggPushdownToTiFlashCompute(t *testing.T) { - var ( - input []string - output []struct { - SQL string - Plan []string - Warning []string - } - ) - planSuiteData := core.GetPlanSuiteData() - planSuiteData.LoadTestCases(t, &input, &output) - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists tbl_15;") - tk.MustExec(`create table tbl_15 (col_89 text (473) collate utf8mb4_bin , - col_90 timestamp default '1976-04-03' , - col_91 tinyint unsigned not null , - col_92 tinyint , - col_93 double not null , - col_94 datetime not null default '1970-06-08' , - col_95 datetime default '2028-02-13' , - col_96 int unsigned not null default 2532480521 , - col_97 char (168) default '') partition by hash (col_91) partitions 4;`) - - tk.MustExec("drop table if exists tbl_16;") - tk.MustExec(`create table tbl_16 (col_98 text (246) not null , - col_99 decimal (30 ,19) , - col_100 mediumint unsigned , - col_101 text (410) collate utf8mb4_bin , - col_102 date not null , - col_103 timestamp not null default '2003-08-27' , - col_104 text (391) not null , - col_105 date default '2010-10-24' , - col_106 text (9) not null,primary key (col_100, col_98(5), col_103), - unique key idx_23 (col_100, col_106 (3), col_101 (3))) partition by hash (col_100) partitions 2;`) - - config.UpdateGlobal(func(conf *config.Config) { - conf.DisaggregatedTiFlash = true - }) - defer config.UpdateGlobal(func(conf *config.Config) { - conf.DisaggregatedTiFlash = false - }) - - dom := domain.GetDomain(tk.Session()) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tblInfo := range db.Tables { - tableName := tblInfo.Name.L - if tableName == "tbl_15" || tableName == "tbl_16" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") - tk.MustExec("set @@tidb_partition_prune_mode = 'static';") - tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash';") - - for i, ts := range input { - testdata.OnRecord(func() { - output[i].SQL = ts - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - } -} diff --git a/types/BUILD.bazel b/types/BUILD.bazel index 454f7972cd071..13d8140953c6f 100644 --- a/types/BUILD.bazel +++ b/types/BUILD.bazel @@ -97,6 +97,7 @@ go_test( ], embed = [":types"], flaky = True, + shard_count = 50, deps = [ "//parser/charset", "//parser/mysql", From 0d860ecb329fe54d8f0f3bba82f78ff648c03cf8 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 15 Feb 2023 02:00:10 +0800 Subject: [PATCH 08/12] *: improve planner/core to make test faster Signed-off-by: Weizhen Wang --- build/nogo_config.json | 1 + planner/core/casetest/BUILD.bazel | 1 + planner/core/casetest/integration_test.go | 1568 ++++++++++++++++ .../core/casetest/partition_pruner_test.go | 322 ++++ planner/core/integration_test.go | 1613 +---------------- planner/core/internal/testkit.go | 15 + planner/core/partition_pruner_test.go | 312 +--- 7 files changed, 1932 insertions(+), 1900 deletions(-) create mode 100644 planner/core/casetest/partition_pruner_test.go diff --git a/build/nogo_config.json b/build/nogo_config.json index 90a5b7cae853f..d0be8e282ec68 100644 --- a/build/nogo_config.json +++ b/build/nogo_config.json @@ -273,6 +273,7 @@ "ineffassign": { "exclude_files": { "external/": "no need to vet third party code", + "cgo/": "no need to vet cgo code", ".*_generated\\.go$": "ignore generated code" } }, diff --git a/planner/core/casetest/BUILD.bazel b/planner/core/casetest/BUILD.bazel index 4e4d9538bb9de..c6e8229ee6854 100644 --- a/planner/core/casetest/BUILD.bazel +++ b/planner/core/casetest/BUILD.bazel @@ -55,6 +55,7 @@ go_test( "//testkit/testmain", "//testkit/testsetup", "//types", + "//util", "//util/collate", "//util/hint", "//util/logutil", diff --git a/planner/core/casetest/integration_test.go b/planner/core/casetest/integration_test.go index 07b0a2b9dd72e..b82ee72c5df4b 100644 --- a/planner/core/casetest/integration_test.go +++ b/planner/core/casetest/integration_test.go @@ -15,6 +15,7 @@ package casetest import ( + "fmt" "strings" "testing" @@ -24,10 +25,13 @@ import ( "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/planner/core/internal" + "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/stmtctx" "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" ) @@ -1826,3 +1830,1567 @@ func TestIssue20710(t *testing.T) { 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 TestPushDownProjectionForTiFlash(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(6,3), name char(128))") + tk.MustExec("analyze table t") + tk.MustExec("set session tidb_allow_mpp=OFF") + + // 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 == "t" { + 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 TestPushDownSelectionForMPP(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(6,3), name char(128))") + tk.MustExec("analyze table t") + + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + tk.MustExec("set @@tidb_isolation_read_engines='tiflash,tidb';") + + 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 TestPushDownProjectionForMPP(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(6,3), name char(128))") + tk.MustExec("analyze table t") + + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_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 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...)) + } +} + +func TestPushDownAggForMPP(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(6,3))") + tk.MustExec("analyze table t") + + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec(" set @@tidb_allow_mpp=1; set @@tidb_broadcast_join_threshold_count = 1; set @@tidb_broadcast_join_threshold_size=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 TestMppUnionAll(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("drop table if exists t1") + tk.MustExec("create table t (a int not null, b int, c varchar(20))") + tk.MustExec("create table t1 (a int, b int not null, c double)") + + // 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 == "t" || tblInfo.Name.L == "t1" { + 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 TestMppJoinDecimal(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("drop table if exists tt") + tk.MustExec("create table t (c1 decimal(8, 5), c2 decimal(9, 5), c3 decimal(9, 4) NOT NULL, c4 decimal(8, 4) NOT NULL, c5 decimal(40, 20))") + tk.MustExec("create table tt (pk int(11) NOT NULL AUTO_INCREMENT primary key,col_varchar_64 varchar(64),col_char_64_not_null char(64) NOT null, col_decimal_30_10_key decimal(30,10), col_tinyint tinyint, col_varchar_key varchar(1), key col_decimal_30_10_key (col_decimal_30_10_key), key col_varchar_key(col_varchar_key));") + tk.MustExec("analyze table t") + tk.MustExec("analyze table tt") + + // 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 == "t" || tblInfo.Name.L == "tt" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@tidb_allow_mpp=1;") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 1") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 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 TestMppJoinExchangeColumnPrune(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 tt") + tk.MustExec("create table t (c1 int, c2 int, c3 int NOT NULL, c4 int NOT NULL, c5 int)") + tk.MustExec("create table tt (b1 int)") + tk.MustExec("analyze table t") + tk.MustExec("analyze table tt") + + // 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 == "t" || tblInfo.Name.L == "tt" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@tidb_allow_mpp=1;") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 1") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 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 TestMppFineGrainedJoinAndAgg(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 tt") + tk.MustExec("create table t (c1 int, c2 int, c3 int NOT NULL, c4 int NOT NULL, c5 int)") + tk.MustExec("create table tt (b1 int)") + tk.MustExec("analyze table t") + tk.MustExec("analyze table tt") + + instances := []string{ + "tiflash,127.0.0.1:3933,127.0.0.1:7777,,", + "tikv,127.0.0.1:11080,127.0.0.1:10080,,", + } + fpName := "github.com/pingcap/tidb/infoschema/mockStoreServerInfo" + fpExpr := `return("` + strings.Join(instances, ";") + `")` + require.NoError(t, failpoint.Enable(fpName, fpExpr)) + defer func() { require.NoError(t, failpoint.Disable(fpName)) }() + fpName2 := "github.com/pingcap/tidb/planner/core/mockTiFlashStreamCountUsingMinLogicalCores" + require.NoError(t, failpoint.Enable(fpName2, `return("8")`)) + defer func() { require.NoError(t, failpoint.Disable(fpName2)) }() + + // 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 == "t" || tblInfo.Name.L == "tt" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@tidb_allow_mpp=1;") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 1") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 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 TestMppAggTopNWithJoin(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(6,3))") + tk.MustExec("analyze table t") + + // 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 == "t" { + 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...)) + } +} + +// 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 TestPushDownProjectionForTiKV(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 real, i int, id int, value decimal(6,3), name char(128), d decimal(6,3), s char(128), t datetime, c bigint as ((a+1)) virtual, e real as ((b+a)))") + tk.MustExec("analyze table t") + tk.MustExec("set session tidb_opt_projection_push_down=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 TestPushDownProjectionForTiFlashCoprocessor(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 real, i int, id int, value decimal(6,3), name char(128), d decimal(6,3), s char(128), t datetime, c bigint as ((a+1)) virtual, e real as ((b+a)))") + tk.MustExec("analyze table t") + tk.MustExec("set session tidb_opt_projection_push_down=1") + + // 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 == "t" { + 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 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 TestInvalidHint(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists tt") + tk.MustExec("create table tt(a int, key(a));") + + var input []string + var output []struct { + SQL string + Plan []string + Warnings []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + warning := "show warnings;" + 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(warning).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 TestRejectSortForMPP(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(6,3), name char(128))") + tk.MustExec("analyze table t") + + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_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 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 TestPushDownGroupConcatToTiFlash(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_0 char(64), col_1 varchar(64) not null, col_2 varchar(1), id int primary key);") + + // 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_isolation_read_engines='tiflash,tidb'; set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + + var input []string + var output []struct { + SQL string + Plan []string + Warning []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...)) + + comment := fmt.Sprintf("case:%v sql:%s", i, tt) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + testdata.OnRecord(func() { + if len(warnings) > 0 { + output[i].Warning = make([]string, len(warnings)) + for j, warning := range warnings { + output[i].Warning[j] = warning.Err.Error() + } + } + }) + if len(output[i].Warning) == 0 { + require.Len(t, warnings, 0, comment) + } else { + require.Len(t, warnings, len(output[i].Warning), comment) + for j, warning := range warnings { + require.Equal(t, stmtctx.WarnLevelWarning, warning.Level, comment) + require.EqualError(t, warning.Err, output[i].Warning[j], comment) + } + } + } +} + +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") + + 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 */)") + tk.MustExec("analyze table partsupp;") + tk.MustExec("analyze table supplier;") + 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} + + h := dom.StatsHandle() + statsTbl1 := h.GetTableStats(tbl1.Meta()) + statsTbl1.Count = 800000 + statsTbl2 := h.GetTableStats(tbl2.Meta()) + statsTbl2.Count = 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 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_18 1.00 root funcs:count(Column#7)->Column#5", + "└─IndexReader_19 1.00 root index:StreamAgg_9", + " └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_17 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo")) +} + +func TestMppVersion(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a bigint, b bigint)") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + 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())) + } + } +} + +// 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())) + } + } +} diff --git a/planner/core/casetest/partition_pruner_test.go b/planner/core/casetest/partition_pruner_test.go new file mode 100644 index 0000000000000..3cbb047495747 --- /dev/null +++ b/planner/core/casetest/partition_pruner_test.go @@ -0,0 +1,322 @@ +// 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 ( + "bytes" + "fmt" + "sort" + "strings" + "testing" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/planner/core/internal" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/stretchr/testify/require" +) + +func TestHashPartitionPruner(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("create database test_partition") + tk.MustExec("use test_partition") + tk.MustExec("drop table if exists t1, t2;") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.MustExec("create table t2(id int, a int, b int, primary key(id, a)) partition by hash(id + a) partitions 10;") + tk.MustExec("create table t1(id int primary key, a int, b int) partition by hash(id) partitions 10;") + tk.MustExec("create table t3(id int, a int, b int, primary key(id, a)) partition by hash(id) partitions 10;") + tk.MustExec("create table t4(d datetime, a int, b int, primary key(d, a)) partition by hash(year(d)) partitions 10;") + tk.MustExec("create table t5(d date, a int, b int, primary key(d, a)) partition by hash(month(d)) partitions 10;") + tk.MustExec("create table t6(a int, b int) partition by hash(a) partitions 3;") + tk.MustExec("create table t7(a int, b int) partition by hash(a + b) partitions 10;") + tk.MustExec("create table t8(a int, b int) partition by hash(a) partitions 6;") + tk.MustExec("create table t9(a bit(1) default null, b int(11) default null) partition by hash(a) partitions 3;") //issue #22619 + tk.MustExec("create table t10(a bigint unsigned) partition BY hash (a);") + + var input []string + var output []struct { + SQL string + Result []string + } + partitionPrunerData := GetPartitionPrunerData() + partitionPrunerData.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 TestListPartitionPruner(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("drop database if exists test_partition;") + tk.MustExec("create database test_partition") + tk.MustExec("use test_partition") + tk.MustExec("set tidb_cost_model_version=2") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.MustExec("set @@session.tidb_enable_list_partition = ON") + tk.MustExec(`set @@session.tidb_regard_null_as_point=false`) + tk.MustExec("create table t1 (id int, a int, b int ) partition by list ( a ) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10,null));") + tk.MustExec("create table t2 (a int, id int, b int) partition by list (a*3 + b - 2*a - b) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10,null));") + tk.MustExec("create table t3 (b int, id int, a int) partition by list columns (a) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10,null));") + tk.MustExec("create table t4 (id int, a int, b int, primary key (a)) partition by list ( a ) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10));") + tk.MustExec("create table t5 (a int, id int, b int, unique key (a,b)) partition by list (a*3 + b - 2*a - b) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10,null));") + tk.MustExec("create table t6 (b int, id int, a int, unique key (a,b)) partition by list columns (a) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10,null));") + tk.MustExec("insert into t1 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") + tk.MustExec("insert into t2 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") + tk.MustExec("insert into t3 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") + tk.MustExec("insert into t4 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10)") + tk.MustExec("insert into t5 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") + tk.MustExec("insert into t6 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") + tk.MustExec(`create table t7 (a int unsigned) partition by list (a)(partition p0 values in (0),partition p1 values in (1),partition pnull values in (null),partition p2 values in (2));`) + tk.MustExec("insert into t7 values (null),(0),(1),(2);") + + // tk2 use to compare the result with normal table. + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("drop database if exists test_partition_2;") + tk2.MustExec("create database test_partition_2") + tk2.MustExec("use test_partition_2") + tk2.MustExec("create table t1 (id int, a int, b int)") + tk2.MustExec("create table t2 (a int, id int, b int)") + tk2.MustExec("create table t3 (b int, id int, a int)") + tk2.MustExec("create table t4 (id int, a int, b int, primary key (a));") + tk2.MustExec("create table t5 (a int, id int, b int, unique key (a,b));") + tk2.MustExec("create table t6 (b int, id int, a int, unique key (a,b));") + tk2.MustExec("insert into t1 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") + tk2.MustExec("insert into t2 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") + tk2.MustExec("insert into t3 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") + tk2.MustExec("insert into t4 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10)") + tk2.MustExec("insert into t5 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") + tk2.MustExec("insert into t6 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") + tk2.MustExec(`create table t7 (a int unsigned);`) + tk2.MustExec("insert into t7 values (null),(0),(1),(2);") + + var input []string + var output []struct { + SQL string + Result []string + Plan []string + } + partitionPrunerData := GetPartitionPrunerData() + partitionPrunerData.LoadTestCases(t, &input, &output) + valid := false + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + result := tk.MustQuery(tt).Sort() + result.Check(testkit.Rows(output[i].Result...)) + // If the query doesn't specified the partition, compare the result with normal table + if !strings.Contains(tt, "partition(") { + result.Check(tk.MustQuery(tt).Sort().Rows()) + valid = true + } + require.True(t, valid) + } +} + +type testTablePartitionInfo struct { + Table string + Partitions string +} + +// getPartitionInfoFromPlan uses to extract table partition information from the plan tree string. Here is an example, the plan is like below: +// +// "Projection_7 80.00 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", +// "└─HashJoin_9 80.00 root CARTESIAN inner join", +// " ├─TableReader_12(Build) 8.00 root partition:p1 data:Selection_11", +// " │ └─Selection_11 8.00 cop[tikv] 1, eq(test_partition.t2.b, 6), in(test_partition.t2.a, 6, 7, 8)", +// " │ └─TableFullScan_10 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", +// " └─TableReader_15(Probe) 10.00 root partition:p0 data:Selection_14", +// " └─Selection_14 10.00 cop[tikv] 1, eq(test_partition.t1.a, 5)", +// " └─TableFullScan_13 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" +// +// The return table partition info is: t1: p0; t2: p1 +func getPartitionInfoFromPlan(plan []string) string { + infos := make([]testTablePartitionInfo, 0, 2) + info := testTablePartitionInfo{} + for _, row := range plan { + partitions := internal.GetFieldValue("partition:", row) + if partitions != "" { + info.Partitions = partitions + continue + } + tbl := internal.GetFieldValue("table:", row) + if tbl != "" { + info.Table = tbl + infos = append(infos, info) + } + } + sort.Slice(infos, func(i, j int) bool { + if infos[i].Table != infos[j].Table { + return infos[i].Table < infos[j].Table + } + return infos[i].Partitions < infos[j].Partitions + }) + buf := bytes.NewBuffer(nil) + for i, info := range infos { + if i > 0 { + buf.WriteString("; ") + } + buf.WriteString(fmt.Sprintf("%v: %v", info.Table, info.Partitions)) + } + return buf.String() +} + +func checkPrunePartitionInfo(c *testing.T, query string, infos1 string, plan []string) { + infos2 := getPartitionInfoFromPlan(plan) + comment := fmt.Sprintf("the query is: %v, the plan is:\n%v", query, strings.Join(plan, "\n")) + require.Equal(c, infos1, infos2, comment) +} + +func TestListColumnsPartitionPruner(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("set tidb_cost_model_version=2") + tk.MustExec("set @@session.tidb_enable_list_partition = ON") + tk.MustExec("drop database if exists test_partition;") + tk.MustExec("create database test_partition") + tk.MustExec("use test_partition") + tk.MustExec("set @@session.tidb_enable_list_partition = ON") + tk.MustExec("create table t1 (id int, a int, b int) partition by list columns (b,a) (partition p0 values in ((1,1),(2,2),(3,3),(4,4),(5,5)), partition p1 values in ((6,6),(7,7),(8,8),(9,9),(10,10),(null,10)));") + tk.MustExec("create table t2 (id int, a int, b int) partition by list columns (id,a,b) (partition p0 values in ((1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5)), partition p1 values in ((6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)));") + tk.MustExec("insert into t1 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,10,null)") + tk.MustExec("insert into t2 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") + + // tk1 use to test partition table with index. + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("set tidb_cost_model_version=2") + tk1.MustExec("drop database if exists test_partition_1;") + tk1.MustExec(`set @@session.tidb_regard_null_as_point=false`) + tk1.MustExec("create database test_partition_1") + tk1.MustExec("use test_partition_1") + tk1.MustExec("set @@session.tidb_enable_list_partition = ON") + tk1.MustExec("create table t1 (id int, a int, b int, unique key (a,b,id)) partition by list columns (b,a) (partition p0 values in ((1,1),(2,2),(3,3),(4,4),(5,5)), partition p1 values in ((6,6),(7,7),(8,8),(9,9),(10,10),(null,10)));") + tk1.MustExec("create table t2 (id int, a int, b int, unique key (a,b,id)) partition by list columns (id,a,b) (partition p0 values in ((1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5)), partition p1 values in ((6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)));") + tk1.MustExec("insert into t1 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,10,null)") + tk1.MustExec("insert into t2 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") + + // tk2 use to compare the result with normal table. + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("set tidb_cost_model_version=2") + tk2.MustExec("drop database if exists test_partition_2;") + tk2.MustExec(`set @@session.tidb_regard_null_as_point=false`) + tk2.MustExec("create database test_partition_2") + tk2.MustExec("use test_partition_2") + tk2.MustExec("create table t1 (id int, a int, b int)") + tk2.MustExec("create table t2 (id int, a int, b int)") + tk2.MustExec("insert into t1 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,10,null)") + tk2.MustExec("insert into t2 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") + + // Default RPC encoding may cause statistics explain result differ and then the test unstable. + tk1.MustExec("set @@tidb_enable_chunk_rpc = on") + + var input []struct { + SQL string + Pruner string + } + var output []struct { + SQL string + Result []string + Plan []string + IndexPlan []string + } + partitionPrunerData := GetPartitionPrunerData() + partitionPrunerData.LoadTestCases(t, &input, &output) + valid := false + for i, tt := range input { + // Test for table without index. + plan := tk.MustQuery("explain format = 'brief' " + tt.SQL) + planTree := testdata.ConvertRowsToStrings(plan.Rows()) + // Test for table with index. + indexPlan := tk1.MustQuery("explain format = 'brief' " + tt.SQL) + indexPlanTree := testdata.ConvertRowsToStrings(indexPlan.Rows()) + testdata.OnRecord(func() { + output[i].SQL = tt.SQL + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt.SQL).Sort().Rows()) + // Test for table without index. + output[i].Plan = planTree + // Test for table with index. + output[i].IndexPlan = indexPlanTree + }) + // compare the plan. + plan.Check(testkit.Rows(output[i].Plan...)) + indexPlan.Check(testkit.Rows(output[i].IndexPlan...)) + + // compare the pruner information. + checkPrunePartitionInfo(t, tt.SQL, tt.Pruner, planTree) + checkPrunePartitionInfo(t, tt.SQL, tt.Pruner, indexPlanTree) + + // compare the result. + result := tk.MustQuery(tt.SQL).Sort() + idxResult := tk1.MustQuery(tt.SQL) + result.Check(idxResult.Sort().Rows()) + result.Check(testkit.Rows(output[i].Result...)) + + // If the query doesn't specified the partition, compare the result with normal table + if !strings.Contains(tt.SQL, "partition(") { + result.Check(tk2.MustQuery(tt.SQL).Sort().Rows()) + valid = true + } + } + require.True(t, valid) +} + +// issue 22079 +func TestRangePartitionPredicatePruner(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Static) + "'") + tk.MustExec("drop database if exists test_partition;") + tk.MustExec("create database test_partition") + tk.MustExec("use test_partition") + tk.MustExec("drop table if exists t") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.MustExec(`create table t (a int(11) default null) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin + partition by range(a) ( + partition p0 values less than (1), + partition p1 values less than (2), + partition p2 values less than (3), + partition p_max values less than (maxvalue));`) + + var input []string + var output []struct { + SQL string + Result []string + } + partitionPrunerData := GetPartitionPrunerData() + partitionPrunerData.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...)) + } +} diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index c01cf4102cf05..7174f734ab334 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -36,12 +36,9 @@ import ( "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/util" "github.com/stretchr/testify/require" ) @@ -2504,33 +2501,6 @@ func TestConditionColPruneInPhysicalUnionScan(t *testing.T) { Check(testkit.Rows("0")) } -func TestInvalidHint(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists tt") - tk.MustExec("create table tt(a int, key(a));") - - var input []string - var output []struct { - SQL string - Plan []string - Warnings []string - } - integrationSuiteData := core.GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - warning := "show warnings;" - 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(warning).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - // Test for issue https://github.com/pingcap/tidb/issues/18320 func TestNonaggregateColumnWithSingleValueInOnlyFullGroupByMode(t *testing.T) { store := testkit.CreateMockStore(t) @@ -2546,46 +2516,6 @@ func TestNonaggregateColumnWithSingleValueInOnlyFullGroupByMode(t *testing.T) { tk.MustQuery("select a from t where a = 1 having count(b) > 0").Check(testkit.Rows("1")) } -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 := 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(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestIssue22040(t *testing.T) { store := testkit.CreateMockStore(t) // #22040 @@ -2606,45 +2536,6 @@ func TestIssue22040(t *testing.T) { } } -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 := 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(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestIssue22071(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2739,258 +2630,6 @@ func TestIssue32428(t *testing.T) { tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows()) // empty result } -func TestPushDownProjectionForTiKV(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 real, i int, id int, value decimal(6,3), name char(128), d decimal(6,3), s char(128), t datetime, c bigint as ((a+1)) virtual, e real as ((b+a)))") - tk.MustExec("analyze table t") - tk.MustExec("set session tidb_opt_projection_push_down=1") - - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestPushDownProjectionForTiFlashCoprocessor(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 real, i int, id int, value decimal(6,3), name char(128), d decimal(6,3), s char(128), t datetime, c bigint as ((a+1)) virtual, e real as ((b+a)))") - tk.MustExec("analyze table t") - tk.MustExec("set session tidb_opt_projection_push_down=1") - - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestPushDownProjectionForTiFlash(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(6,3), name char(128))") - tk.MustExec("analyze table t") - tk.MustExec("set session tidb_allow_mpp=OFF") - - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestPushDownSelectionForMPP(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(6,3), name char(128))") - tk.MustExec("analyze table t") - - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") - tk.MustExec("set @@tidb_isolation_read_engines='tiflash,tidb';") - - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestPushDownProjectionForMPP(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(6,3), name char(128))") - tk.MustExec("analyze table t") - - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") - - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -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 := 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(tt).Rows()) - }) - tk.MustQuery(tt).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 := 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].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 TestDeleteStmt(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -3023,357 +2662,6 @@ func TestIndexMergeConstantTrue(t *testing.T) { tk.MustExec("delete /*+ use_index_merge(t) */ FROM t WHERE a=1 OR (a<2 and b<2)") } -func TestPushDownAggForMPP(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(6,3))") - tk.MustExec("analyze table t") - - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec(" set @@tidb_allow_mpp=1; set @@tidb_broadcast_join_threshold_count = 1; set @@tidb_broadcast_join_threshold_size=1;") - - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestMppUnionAll(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("drop table if exists t1") - tk.MustExec("create table t (a int not null, b int, c varchar(20))") - tk.MustExec("create table t1 (a int, b int not null, c double)") - - // 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 == "t" || tblInfo.Name.L == "t1" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestMppJoinDecimal(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("drop table if exists tt") - tk.MustExec("create table t (c1 decimal(8, 5), c2 decimal(9, 5), c3 decimal(9, 4) NOT NULL, c4 decimal(8, 4) NOT NULL, c5 decimal(40, 20))") - tk.MustExec("create table tt (pk int(11) NOT NULL AUTO_INCREMENT primary key,col_varchar_64 varchar(64),col_char_64_not_null char(64) NOT null, col_decimal_30_10_key decimal(30,10), col_tinyint tinyint, col_varchar_key varchar(1), key col_decimal_30_10_key (col_decimal_30_10_key), key col_varchar_key(col_varchar_key));") - tk.MustExec("analyze table t") - tk.MustExec("analyze table tt") - - // 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 == "t" || tblInfo.Name.L == "tt" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@tidb_allow_mpp=1;") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 1") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 1") - - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestMppJoinExchangeColumnPrune(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 tt") - tk.MustExec("create table t (c1 int, c2 int, c3 int NOT NULL, c4 int NOT NULL, c5 int)") - tk.MustExec("create table tt (b1 int)") - tk.MustExec("analyze table t") - tk.MustExec("analyze table tt") - - // 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 == "t" || tblInfo.Name.L == "tt" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@tidb_allow_mpp=1;") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 1") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 1") - - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestMppFineGrainedJoinAndAgg(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 tt") - tk.MustExec("create table t (c1 int, c2 int, c3 int NOT NULL, c4 int NOT NULL, c5 int)") - tk.MustExec("create table tt (b1 int)") - tk.MustExec("analyze table t") - tk.MustExec("analyze table tt") - - instances := []string{ - "tiflash,127.0.0.1:3933,127.0.0.1:7777,,", - "tikv,127.0.0.1:11080,127.0.0.1:10080,,", - } - fpName := "github.com/pingcap/tidb/infoschema/mockStoreServerInfo" - fpExpr := `return("` + strings.Join(instances, ";") + `")` - require.NoError(t, failpoint.Enable(fpName, fpExpr)) - defer func() { require.NoError(t, failpoint.Disable(fpName)) }() - fpName2 := "github.com/pingcap/tidb/planner/core/mockTiFlashStreamCountUsingMinLogicalCores" - require.NoError(t, failpoint.Enable(fpName2, `return("8")`)) - defer func() { require.NoError(t, failpoint.Disable(fpName2)) }() - - // 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 == "t" || tblInfo.Name.L == "tt" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@tidb_allow_mpp=1;") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 1") - tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 1") - - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestMppAggTopNWithJoin(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(6,3))") - tk.MustExec("analyze table t") - - // 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 == "t" { - 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 := 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(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 := 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(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 := 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(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 := 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(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestIndexMergeTableFilter(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -3515,31 +2803,6 @@ func TestIndexMergeClusterIndex(t *testing.T) { )) } -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 := 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()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestIssue23736(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -3608,32 +2871,6 @@ func TestIssue23839(t *testing.T) { tk.Exec("explain SELECT OUTR . col2 AS X FROM (SELECT INNR . col1 as col1, SUM( INNR . col2 ) as col2 FROM (SELECT INNR . `col_int_not_null` + 1 as col1, INNR . `pk` as col2 FROM BB AS INNR) AS INNR GROUP BY col1) AS OUTR2 INNER JOIN (SELECT INNR . col1 as col1, MAX( INNR . col2 ) as col2 FROM (SELECT INNR . `col_int_not_null` + 1 as col1, INNR . `pk` as col2 FROM BB AS INNR) AS INNR GROUP BY col1) AS OUTR ON OUTR2.col1 = OUTR.col1 GROUP BY OUTR . col1, OUTR2 . col1 HAVING X <> 'b'") } -// 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 := 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()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestIssue24281(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -3737,33 +2974,6 @@ func TestConflictReadFromStorage(t *testing.T) { tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 Storage hints are conflict, you can only specify one storage type of table test.t")) } -// 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 := 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()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestIssue27167(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -3814,46 +3024,6 @@ func TestIssue25300(t *testing.T) { tk.MustGetErrCode(`(select a from t) union ( select b from t) union select 'a' except select 'd';`, mysql.ErrCantAggregateNcollations) } -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 := 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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } -} - func TestSelectIgnoreTemporaryTableInView(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -3883,32 +3053,6 @@ func TestSelectIgnoreTemporaryTableInView(t *testing.T) { tk.MustQuery("select * from v5").Check(testkit.Rows("1 2", "3 4")) } -// 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 := 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()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestIssue26250(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -4104,171 +3248,40 @@ func TestLimitPushDown(t *testing.T) { tk.MustQuery(`explain format=brief select a from t order by a desc limit 11`).Check(testkit.Rows( `TopN 1.00 root test.t.a:desc, offset:0, count:11`, - `└─TableReader 1.00 root data:TableFullScan`, - ` └─TableFullScan 1.00 cop[tikv] table:t keep order:false`)) - - tk.MustQuery(`explain format=brief select /*+ limit_to_cop() */ a from t order by a desc limit 11`).Check(testkit.Rows( - `TopN 1.00 root test.t.a:desc, offset:0, count:11`, - `└─TableReader 1.00 root data:TopN`, - ` └─TopN 1.00 cop[tikv] test.t.a:desc, offset:0, count:11`, - ` └─TableFullScan 1.00 cop[tikv] table:t keep order:false`)) -} - -func TestIssue26559(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t(a timestamp, b datetime);") - tk.MustExec("insert into t values('2020-07-29 09:07:01', '2020-07-27 16:57:36');") - tk.MustQuery("select greatest(a, b) from t union select null;").Sort().Check(testkit.Rows("2020-07-29 09:07:01", "")) -} - -func TestIssue29503(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - defer config.RestoreFunc()() - config.UpdateGlobal(func(conf *config.Config) { - conf.Status.RecordQPSbyDB = true - }) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a int);") - require.NoError(t, tk.ExecToErr("create binding for select 1 using select 1;")) - require.NoError(t, tk.ExecToErr("create binding for select a from t using select a from t;")) - res := tk.MustQuery("show session bindings;") - require.Len(t, res.Rows(), 2) -} - -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 := 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 = '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") + `└─TableReader 1.00 root data:TableFullScan`, + ` └─TableFullScan 1.00 cop[tikv] table:t keep order:false`)) - var input []string - var output []struct { - SQL string - Plan []string - Warnings []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 = '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...)) - } + tk.MustQuery(`explain format=brief select /*+ limit_to_cop() */ a from t order by a desc limit 11`).Check(testkit.Rows( + `TopN 1.00 root test.t.a:desc, offset:0, count:11`, + `└─TableReader 1.00 root data:TopN`, + ` └─TopN 1.00 cop[tikv] test.t.a:desc, offset:0, count:11`, + ` └─TableFullScan 1.00 cop[tikv] table:t keep order:false`)) } -func TestPreferRangeScanForUnsignedIntHandle(t *testing.T) { +func TestIssue26559(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 := core.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...)) - } + tk.MustExec("create table t(a timestamp, b datetime);") + tk.MustExec("insert into t values('2020-07-29 09:07:01', '2020-07-27 16:57:36');") + tk.MustQuery("select greatest(a, b) from t union select null;").Sort().Check(testkit.Rows("2020-07-29 09:07:01", "")) } -func TestIssue27083(t *testing.T) { +func TestIssue29503(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") + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.Status.RecordQPSbyDB = true + }) - var input []string - var output []struct { - SQL string - Plan []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()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - } + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int);") + require.NoError(t, tk.ExecToErr("create binding for select 1 using select 1;")) + require.NoError(t, tk.ExecToErr("create binding for select a from t using select a from t;")) + res := tk.MustQuery("show session bindings;") + require.Len(t, res.Rows(), 2) } func TestIssues27130(t *testing.T) { @@ -4378,97 +3391,6 @@ func TestTemporaryTableForCte(t *testing.T) { rows.Check(testkit.Rows("1", "2", "3", "4", "5")) } -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 := core.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 TestPushDownGroupConcatToTiFlash(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_0 char(64), col_1 varchar(64) not null, col_2 varchar(1), id int primary key);") - - // 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_isolation_read_engines='tiflash,tidb'; set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") - - var input []string - var output []struct { - SQL string - Plan []string - Warning []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - - comment := fmt.Sprintf("case:%v sql:%s", i, tt) - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - testdata.OnRecord(func() { - if len(warnings) > 0 { - output[i].Warning = make([]string, len(warnings)) - for j, warning := range warnings { - output[i].Warning[j] = warning.Err.Error() - } - } - }) - if len(output[i].Warning) == 0 { - require.Len(t, warnings, 0, comment) - } else { - require.Len(t, warnings, len(output[i].Warning), comment) - for j, warning := range warnings { - require.Equal(t, stmtctx.WarnLevelWarning, warning.Level, comment) - require.EqualError(t, warning.Err, output[i].Warning[j], comment) - } - } - } -} - func TestIssue27797(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -4562,100 +3484,6 @@ func TestIssue28154(t *testing.T) { result.Check(testkit.Rows("abc")) } -func TestRejectSortForMPP(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(6,3), name char(128))") - tk.MustExec("analyze table t") - - // 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 == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") - - var input []string - var output []struct { - SQL string - Plan []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(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.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 := core.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 TestIssues29711(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -4722,52 +3550,6 @@ func TestIssue30094(t *testing.T) { )) } -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 := 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].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 TestIssue29705(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -4835,53 +3617,6 @@ func TestIndexMergeWarning(t *testing.T) { tk.MustQuery("show warnings").Check(testkit.Rows(warningMsg)) } -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 := 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].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 TestIssue20510(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -5059,170 +3794,6 @@ func TestAggPushToCopForCachedTable(t *testing.T) { tk.MustExec("drop table if exists t31202") } -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") - - 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 := core.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 */)") - tk.MustExec("analyze table partsupp;") - tk.MustExec("analyze table supplier;") - 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} - - h := dom.StatsHandle() - statsTbl1 := h.GetTableStats(tbl1.Meta()) - statsTbl1.Count = 800000 - statsTbl2 := h.GetTableStats(tbl2.Meta()) - statsTbl2.Count = 10000 - var input []string - var output []struct { - SQL string - Plan []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(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 := 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(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 := 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(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestTiFlashFineGrainedShuffleWithMaxTiFlashThreads(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) @@ -6367,40 +4938,6 @@ func TestExplainAnalyzeDMLCommit(t *testing.T) { tk.MustQuery("select * from t").Check(testkit.Rows()) } -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 := core.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 TestPlanCacheForIndexJoinRangeFallback(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -6471,62 +5008,6 @@ func TestOuterJoinEliminationForIssue18216(t *testing.T) { tk.MustQuery("select group_concat(c order by (select group_concat(c order by c) from t2 where a=t1.a), c desc) from t1;").Check(testkit.Rows("2,1,4,3")) } -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 := 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...)) - } - - // 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_18 1.00 root funcs:count(Column#7)->Column#5", - "└─IndexReader_19 1.00 root index:StreamAgg_9", - " └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#7", - " └─IndexRangeScan_17 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo")) -} - func TestAutoIncrementCheckWithCheckConstraint(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -6538,54 +5019,6 @@ func TestAutoIncrementCheckWithCheckConstraint(t *testing.T) { )`) } -func TestMppVersion(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a bigint, b bigint)") - tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") - tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") - - // Create virtual tiflash replica info. - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - require.True(t, exists) - for _, tblInfo := range db.Tables { - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - var input []string - var output []struct { - SQL string - Plan []string - Warn []string - } - integrationSuiteData := core.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())) - } - } -} - // https://github.com/pingcap/tidb/issues/36888. func TestIssue36888(t *testing.T) { store := testkit.CreateMockStore(t) diff --git a/planner/core/internal/testkit.go b/planner/core/internal/testkit.go index 214fc89b01b04..8e8d70dc0f9d1 100644 --- a/planner/core/internal/testkit.go +++ b/planner/core/internal/testkit.go @@ -16,6 +16,7 @@ package internal import ( "fmt" + "strings" "testing" "github.com/pingcap/kvproto/pkg/metapb" @@ -74,3 +75,17 @@ func WithMockTiFlash(nodes int) mockstore.MockTiKVStoreOption { mockstore.WithStoreType(mockstore.EmbedUnistore), ) } + +// GetFieldValue is to get field value. +func GetFieldValue(prefix, row string) string { + if idx := strings.Index(row, prefix); idx > 0 { + start := idx + len(prefix) + end := strings.Index(row[start:], " ") + if end > 0 { + value := row[start : start+end] + value = strings.Trim(value, ",") + return value + } + } + return "" +} diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index 166f251b53fc3..cf35e7250867c 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -15,56 +15,17 @@ package core_test import ( - "bytes" "fmt" - "sort" "strings" "testing" "github.com/pingcap/failpoint" - plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/planner/core/internal" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" "github.com/stretchr/testify/require" ) -func TestHashPartitionPruner(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("create database test_partition") - tk.MustExec("use test_partition") - tk.MustExec("drop table if exists t1, t2;") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly - tk.MustExec("create table t2(id int, a int, b int, primary key(id, a)) partition by hash(id + a) partitions 10;") - tk.MustExec("create table t1(id int primary key, a int, b int) partition by hash(id) partitions 10;") - tk.MustExec("create table t3(id int, a int, b int, primary key(id, a)) partition by hash(id) partitions 10;") - tk.MustExec("create table t4(d datetime, a int, b int, primary key(d, a)) partition by hash(year(d)) partitions 10;") - tk.MustExec("create table t5(d date, a int, b int, primary key(d, a)) partition by hash(month(d)) partitions 10;") - tk.MustExec("create table t6(a int, b int) partition by hash(a) partitions 3;") - tk.MustExec("create table t7(a int, b int) partition by hash(a + b) partitions 10;") - tk.MustExec("create table t8(a int, b int) partition by hash(a) partitions 6;") - tk.MustExec("create table t9(a bit(1) default null, b int(11) default null) partition by hash(a) partitions 3;") //issue #22619 - tk.MustExec("create table t10(a bigint unsigned) partition BY hash (a);") - - var input []string - var output []struct { - SQL string - Result []string - } - partitionPrunerData := plannercore.GetPartitionPrunerData() - partitionPrunerData.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 TestRangeColumnPartitionPruningForIn(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -165,7 +126,7 @@ func TestRangeColumnPartitionPruningForInString(t *testing.T) { planStrings := testdata.ConvertRowsToStrings(res.Rows()) partitions := []string{} for _, s := range planStrings { - parts := getFieldValue("partition:", s) + parts := internal.GetFieldValue("partition:", s) if parts != "" { partitions = append(partitions, strings.Split(parts, ",")...) } @@ -253,242 +214,6 @@ func TestRangeColumnPartitionPruningForInString(t *testing.T) { checkColumnStringPruningTests(tests) } -func TestListPartitionPruner(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("drop database if exists test_partition;") - tk.MustExec("create database test_partition") - tk.MustExec("use test_partition") - tk.MustExec("set tidb_cost_model_version=2") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly - tk.MustExec("set @@session.tidb_enable_list_partition = ON") - tk.MustExec(`set @@session.tidb_regard_null_as_point=false`) - tk.MustExec("create table t1 (id int, a int, b int ) partition by list ( a ) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10,null));") - tk.MustExec("create table t2 (a int, id int, b int) partition by list (a*3 + b - 2*a - b) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10,null));") - tk.MustExec("create table t3 (b int, id int, a int) partition by list columns (a) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10,null));") - tk.MustExec("create table t4 (id int, a int, b int, primary key (a)) partition by list ( a ) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10));") - tk.MustExec("create table t5 (a int, id int, b int, unique key (a,b)) partition by list (a*3 + b - 2*a - b) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10,null));") - tk.MustExec("create table t6 (b int, id int, a int, unique key (a,b)) partition by list columns (a) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10,null));") - tk.MustExec("insert into t1 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") - tk.MustExec("insert into t2 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") - tk.MustExec("insert into t3 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") - tk.MustExec("insert into t4 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10)") - tk.MustExec("insert into t5 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") - tk.MustExec("insert into t6 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") - tk.MustExec(`create table t7 (a int unsigned) partition by list (a)(partition p0 values in (0),partition p1 values in (1),partition pnull values in (null),partition p2 values in (2));`) - tk.MustExec("insert into t7 values (null),(0),(1),(2);") - - // tk2 use to compare the result with normal table. - tk2 := testkit.NewTestKit(t, store) - tk2.MustExec("drop database if exists test_partition_2;") - tk2.MustExec("create database test_partition_2") - tk2.MustExec("use test_partition_2") - tk2.MustExec("create table t1 (id int, a int, b int)") - tk2.MustExec("create table t2 (a int, id int, b int)") - tk2.MustExec("create table t3 (b int, id int, a int)") - tk2.MustExec("create table t4 (id int, a int, b int, primary key (a));") - tk2.MustExec("create table t5 (a int, id int, b int, unique key (a,b));") - tk2.MustExec("create table t6 (b int, id int, a int, unique key (a,b));") - tk2.MustExec("insert into t1 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") - tk2.MustExec("insert into t2 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") - tk2.MustExec("insert into t3 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") - tk2.MustExec("insert into t4 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10)") - tk2.MustExec("insert into t5 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") - tk2.MustExec("insert into t6 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") - tk2.MustExec(`create table t7 (a int unsigned);`) - tk2.MustExec("insert into t7 values (null),(0),(1),(2);") - - var input []string - var output []struct { - SQL string - Result []string - Plan []string - } - partitionPrunerData := plannercore.GetPartitionPrunerData() - partitionPrunerData.LoadTestCases(t, &input, &output) - valid := false - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - result := tk.MustQuery(tt).Sort() - result.Check(testkit.Rows(output[i].Result...)) - // If the query doesn't specified the partition, compare the result with normal table - if !strings.Contains(tt, "partition(") { - result.Check(tk.MustQuery(tt).Sort().Rows()) - valid = true - } - require.True(t, valid) - } -} - -func TestListColumnsPartitionPruner(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("set tidb_cost_model_version=2") - tk.MustExec("set @@session.tidb_enable_list_partition = ON") - tk.MustExec("drop database if exists test_partition;") - tk.MustExec("create database test_partition") - tk.MustExec("use test_partition") - tk.MustExec("set @@session.tidb_enable_list_partition = ON") - tk.MustExec("create table t1 (id int, a int, b int) partition by list columns (b,a) (partition p0 values in ((1,1),(2,2),(3,3),(4,4),(5,5)), partition p1 values in ((6,6),(7,7),(8,8),(9,9),(10,10),(null,10)));") - tk.MustExec("create table t2 (id int, a int, b int) partition by list columns (id,a,b) (partition p0 values in ((1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5)), partition p1 values in ((6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)));") - tk.MustExec("insert into t1 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,10,null)") - tk.MustExec("insert into t2 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") - - // tk1 use to test partition table with index. - tk1 := testkit.NewTestKit(t, store) - tk1.MustExec("set tidb_cost_model_version=2") - tk1.MustExec("drop database if exists test_partition_1;") - tk1.MustExec(`set @@session.tidb_regard_null_as_point=false`) - tk1.MustExec("create database test_partition_1") - tk1.MustExec("use test_partition_1") - tk1.MustExec("set @@session.tidb_enable_list_partition = ON") - tk1.MustExec("create table t1 (id int, a int, b int, unique key (a,b,id)) partition by list columns (b,a) (partition p0 values in ((1,1),(2,2),(3,3),(4,4),(5,5)), partition p1 values in ((6,6),(7,7),(8,8),(9,9),(10,10),(null,10)));") - tk1.MustExec("create table t2 (id int, a int, b int, unique key (a,b,id)) partition by list columns (id,a,b) (partition p0 values in ((1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5)), partition p1 values in ((6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)));") - tk1.MustExec("insert into t1 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,10,null)") - tk1.MustExec("insert into t2 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") - - // tk2 use to compare the result with normal table. - tk2 := testkit.NewTestKit(t, store) - tk2.MustExec("set tidb_cost_model_version=2") - tk2.MustExec("drop database if exists test_partition_2;") - tk2.MustExec(`set @@session.tidb_regard_null_as_point=false`) - tk2.MustExec("create database test_partition_2") - tk2.MustExec("use test_partition_2") - tk2.MustExec("create table t1 (id int, a int, b int)") - tk2.MustExec("create table t2 (id int, a int, b int)") - tk2.MustExec("insert into t1 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,10,null)") - tk2.MustExec("insert into t2 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") - - // Default RPC encoding may cause statistics explain result differ and then the test unstable. - tk1.MustExec("set @@tidb_enable_chunk_rpc = on") - - var input []struct { - SQL string - Pruner string - } - var output []struct { - SQL string - Result []string - Plan []string - IndexPlan []string - } - partitionPrunerData := plannercore.GetPartitionPrunerData() - partitionPrunerData.LoadTestCases(t, &input, &output) - valid := false - for i, tt := range input { - // Test for table without index. - plan := tk.MustQuery("explain format = 'brief' " + tt.SQL) - planTree := testdata.ConvertRowsToStrings(plan.Rows()) - // Test for table with index. - indexPlan := tk1.MustQuery("explain format = 'brief' " + tt.SQL) - indexPlanTree := testdata.ConvertRowsToStrings(indexPlan.Rows()) - testdata.OnRecord(func() { - output[i].SQL = tt.SQL - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt.SQL).Sort().Rows()) - // Test for table without index. - output[i].Plan = planTree - // Test for table with index. - output[i].IndexPlan = indexPlanTree - }) - // compare the plan. - plan.Check(testkit.Rows(output[i].Plan...)) - indexPlan.Check(testkit.Rows(output[i].IndexPlan...)) - - // compare the pruner information. - checkPrunePartitionInfo(t, tt.SQL, tt.Pruner, planTree) - checkPrunePartitionInfo(t, tt.SQL, tt.Pruner, indexPlanTree) - - // compare the result. - result := tk.MustQuery(tt.SQL).Sort() - idxResult := tk1.MustQuery(tt.SQL) - result.Check(idxResult.Sort().Rows()) - result.Check(testkit.Rows(output[i].Result...)) - - // If the query doesn't specified the partition, compare the result with normal table - if !strings.Contains(tt.SQL, "partition(") { - result.Check(tk2.MustQuery(tt.SQL).Sort().Rows()) - valid = true - } - } - require.True(t, valid) -} - -func checkPrunePartitionInfo(c *testing.T, query string, infos1 string, plan []string) { - infos2 := getPartitionInfoFromPlan(plan) - comment := fmt.Sprintf("the query is: %v, the plan is:\n%v", query, strings.Join(plan, "\n")) - require.Equal(c, infos1, infos2, comment) -} - -type testTablePartitionInfo struct { - Table string - Partitions string -} - -// getPartitionInfoFromPlan uses to extract table partition information from the plan tree string. Here is an example, the plan is like below: -// -// "Projection_7 80.00 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", -// "└─HashJoin_9 80.00 root CARTESIAN inner join", -// " ├─TableReader_12(Build) 8.00 root partition:p1 data:Selection_11", -// " │ └─Selection_11 8.00 cop[tikv] 1, eq(test_partition.t2.b, 6), in(test_partition.t2.a, 6, 7, 8)", -// " │ └─TableFullScan_10 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", -// " └─TableReader_15(Probe) 10.00 root partition:p0 data:Selection_14", -// " └─Selection_14 10.00 cop[tikv] 1, eq(test_partition.t1.a, 5)", -// " └─TableFullScan_13 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" -// -// The return table partition info is: t1: p0; t2: p1 -func getPartitionInfoFromPlan(plan []string) string { - infos := make([]testTablePartitionInfo, 0, 2) - info := testTablePartitionInfo{} - for _, row := range plan { - partitions := getFieldValue("partition:", row) - if partitions != "" { - info.Partitions = partitions - continue - } - tbl := getFieldValue("table:", row) - if tbl != "" { - info.Table = tbl - infos = append(infos, info) - } - } - sort.Slice(infos, func(i, j int) bool { - if infos[i].Table != infos[j].Table { - return infos[i].Table < infos[j].Table - } - return infos[i].Partitions < infos[j].Partitions - }) - buf := bytes.NewBuffer(nil) - for i, info := range infos { - if i > 0 { - buf.WriteString("; ") - } - buf.WriteString(fmt.Sprintf("%v: %v", info.Table, info.Partitions)) - } - return buf.String() -} - -func getFieldValue(prefix, row string) string { - if idx := strings.Index(row, prefix); idx > 0 { - start := idx + len(prefix) - end := strings.Index(row[start:], " ") - if end > 0 { - value := row[start : start+end] - value = strings.Trim(value, ",") - return value - } - } - return "" -} - func TestListColumnsPartitionPrunerRandom(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -653,39 +378,6 @@ partition by range (a) ( tk.MustQuery("select * from t3 where not (a != 1)").Check(testkit.Rows("1")) } -// issue 22079 -func TestRangePartitionPredicatePruner(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Static) + "'") - tk.MustExec("drop database if exists test_partition;") - tk.MustExec("create database test_partition") - tk.MustExec("use test_partition") - tk.MustExec("drop table if exists t") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly - tk.MustExec(`create table t (a int(11) default null) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin - partition by range(a) ( - partition p0 values less than (1), - partition p1 values less than (2), - partition p2 values less than (3), - partition p_max values less than (maxvalue));`) - - var input []string - var output []struct { - SQL string - Result []string - } - partitionPrunerData := plannercore.GetPartitionPrunerData() - partitionPrunerData.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 TestHashPartitionPruning(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) From 7b595dc6c28b1e6950c70247c3d3a50409e60883 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 15 Feb 2023 10:06:55 +0800 Subject: [PATCH 09/12] *: improve planner/core to make test faster Signed-off-by: Weizhen Wang --- planner/core/casetest/BUILD.bazel | 1 + 1 file changed, 1 insertion(+) diff --git a/planner/core/casetest/BUILD.bazel b/planner/core/casetest/BUILD.bazel index c6e8229ee6854..5b4152634bddc 100644 --- a/planner/core/casetest/BUILD.bazel +++ b/planner/core/casetest/BUILD.bazel @@ -12,6 +12,7 @@ go_test( "integration_partition_test.go", "integration_test.go", "main_test.go", + "partition_pruner_test.go", "physical_plan_test.go", "plan_test.go", "point_get_plan_test.go", From 7956757cb58d501221a293e613d74dcd835a54a5 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 15 Feb 2023 12:01:40 +0800 Subject: [PATCH 10/12] *: bazel upgrade shard_count Signed-off-by: Weizhen Wang --- .../testdata/index_merge_suite_in.json | 144 -- .../testdata/index_merge_suite_out.json | 1188 ----------------- 2 files changed, 1332 deletions(-) delete mode 100644 planner/core/casetest/testdata/index_merge_suite_in.json delete mode 100644 planner/core/casetest/testdata/index_merge_suite_out.json diff --git a/planner/core/casetest/testdata/index_merge_suite_in.json b/planner/core/casetest/testdata/index_merge_suite_in.json deleted file mode 100644 index d660364305397..0000000000000 --- a/planner/core/casetest/testdata/index_merge_suite_in.json +++ /dev/null @@ -1,144 +0,0 @@ -[ - { - "name": "TestEnforceMVIndex", - "cases": [ - "select /*+ use_index(t, kj) */ * from t", - "select /*+ use_index(t, kj) */ a from t", - "select /*+ use_index(t, kj) */ * from t where a<10", - "select /*+ use_index(t, kj) */ * from t where (1 member of (j))", - "select /*+ use_index(t, kj) */ * from t where (1 member of (j)) and a=10", - "select /*+ use_index(t, kj) */ * from t where (1 member of (j)) or a=10", - "select /*+ use_index_merge(t, kj) */ * from t", - "select /*+ use_index_merge(t, kj) */ a from t", - "select /*+ use_index_merge(t, kj) */ * from t where a<10", - "select /*+ use_index_merge(t, kj) */ * from t where (1 member of (j)) or a=10" - ] - }, - { - "name": "TestIndexMergeJSONMemberOf", - "cases": [ - "select /*+ use_index_merge(t, j0_0) */ * from t where (1 member of (j0->'$.path0'))", - "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and a<10", - "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.XXX')) and a<10", - "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and (2 member of (j1)) and a<10", - "select /*+ use_index(t, j0_0) */ * from t where (1 member of (j0->'$.path0'))", - "select /*+ use_index(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and a<10", - "select * from t use index(j0_0) where (1 member of (j0->'$.path0'))", - "select * from t use index(j0_1) where (1 member of (j0->'$.path1')) and a<10", - "select * from t force index(j0_0) where (1 member of (j0->'$.path0'))", - "select * from t force index(j0_1) where (1 member of (j0->'$.path1')) and a<10", - "select /*+ use_index_merge(t, j1) */ * from t where (1 member of (j0->'$.path1')) and (2 member of (j1)) and a<10", - "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '[1, 2, 3]')", - "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '[1, 2, 3]')", - "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('[1, 2, 3]', (j0->'$.path0'))", - "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '[1, 2, 3]') and a<10", - "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '[1, 2, 3]') and a<10", - "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('[1, 2, 3]', (j0->'$.path0')) and a<10", - "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '1')", - "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '1')", - "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('1', (j0->'$.path0'))", - "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '1') and a<10", - "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '1') and a<10", - "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('1', (j0->'$.path0')) and a<10", - "select /*+ use_index_merge(t, j0_string) */ * from t where (\"a\" member of (j0->'$.path_string'))", - "select /*+ use_index_merge(t, j0_string) */ * from t where (\"a\" member of (j0->'$.path_string')) and a<10", - "select /*+ use_index_merge(t, j0_string) */ * from t where json_contains((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]')", - "select /*+ use_index_merge(t, j0_string) */ * from t where json_contains((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]') and a<10", - "select /*+ use_index_merge(t, j0_string) */ * from t where json_overlaps((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]')", - "select /*+ use_index_merge(t, j0_string) */ * from t where json_overlaps((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]') and a<10", - "select /*+ use_index_merge(t, j0_date) */ * from t where (\"2023-01-01\" member of (j0->'$.path_date'))", - "select /*+ use_index_merge(t, j0_date) */ * from t where (\"2023-01-01\" member of (j0->'$.path_date')) and a<10", - "select /*+ use_index_merge(t, j0_date) */ * from t where json_contains((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date)))", - "select /*+ use_index_merge(t, j0_date) */ * from t where json_contains((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date))) and a<10", - "select /*+ use_index_merge(t, j0_date) */ * from t where json_overlaps((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date)))", - "select /*+ use_index_merge(t, j0_date) */ * from t where json_overlaps((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date))) and a<10" - ] - }, - { - "name": "TestCompositeMVIndex", - "cases": [ - "select /*+ use_index_merge(t, idx) */ * from t where a=1 and b=2 and (3 member of (j)) and c=4", - "select /*+ use_index_merge(t, idx) */ * from t where a=1 and b=2 and (3 member of (j))", - "select /*+ use_index_merge(t, idx) */ * from t where a=1 and b=2", - "select /*+ use_index_merge(t, idx) */ * from t where a=1", - "select /*+ use_index_merge(t, idx2) */ * from t where a=1 and b=2 and ('3' member of (j->'$.str')) and c=4", - "select /*+ use_index_merge(t, idx2) */ * from t where a=1 and b=2 and ('3' member of (j->'$.str'))", - "select /*+ use_index_merge(t, idx2) */ * from t where a=1 and b=2", - "select /*+ use_index_merge(t, idx2) */ * from t where a=1", - "select /*+ use_index(t, idx) */ * from t where a=1 and b=2 and (3 member of (j)) and c=4", - "select * from t use index(idx) where a=1 and b=2 and (3 member of (j))", - "select /*+ use_index(t, idx) */ * from t where a=1 and b=2", - "select * from t use index(idx) where a=1", - "select * from t force index(idx) where a=1 and b=2 and (3 member of (j))", - "select * from t force index(idx) where a=1" - ] - }, - { - "name": "TestDNFOnMVIndex", - "cases": [ - "select /*+ use_index_merge(t, idx1) */ * from t where (1 member of (j)) or (2 member of (j))", - "select /*+ use_index_merge(t, idx1) */ * from t where ((1 member of (j)) or (2 member of (j))) and (a > 10)", - "select /*+ use_index_merge(t, idx1) */ * from t where (json_overlaps(j, '[1, 2]')) or (json_overlaps(j, '[3, 4]'))", - "select /*+ use_index_merge(t, idx1) */ * from t where ((json_overlaps(j, '[1, 2]')) or (json_overlaps(j, '[3, 4]'))) and (a > 10)", - "select /*+ use_index_merge(t, idx1) */ * from t where (json_contains(j, '[1, 2]')) or (json_contains(j, '[3, 4]'))", - "select /*+ use_index_merge(t, idx2) */ * from t where (a=1 and b=2 and (3 member of (j))) or (a=11 and b=12 and (13 member of (j)))", - "select /*+ use_index_merge(t, idx2) */ * from t where (a=1 and b=2 and (3 member of (j))) or (a=11 and b=12 and (13 member of (j)) and c=14)", - "select /*+ use_index_merge(t, idx2) */ * from t where ((a=1 and b=2 and (3 member of (j))) or (a=11 and b=12 and (13 member of (j)))) and (c > 10)" - ] - }, - { - "name": "TestMVIndexSelection", - "cases": [ - "select (j->'$.int') from t where (1 member of (j->'$.int'))", - "select * from t where (1 member of (j->'$.int'))", - "select * from t where (1 member of (j->'$.int')) and a<10", - "select (j->'$.int') from t where json_contains((j->'$.int'), '[1, 2, 3]')", - "select * from t where json_contains((j->'$.int'), '[1, 2, 3]')", - "select * from t where json_contains((j->'$.int'), '[1, 2, 3]') and a<10", - "select (j->'$.int') from t where json_overlaps((j->'$.int'), '[1, 2, 3]')", - "select * from t where json_overlaps((j->'$.int'), '[1, 2, 3]')", - "select * from t where json_overlaps((j->'$.int'), '[1, 2, 3]') and a<10" - ] - }, - { - "name": "TestIndexMergePathGeneration", - "cases": [ - "select * from t", - "select * from t where c < 1", - "select * from t where c < 1 or f > 2", - "select * from t where (c < 1 or f > 2) and (c > 5 or f < 7)", - "select * from t where (c < 1 or f > 2) and (c > 5 or f < 7) and (c < 1 or g > 2)", - "select * from t where (c < 1 or f > 2) and (c > 5 or f < 7) and (e < 1 or f > 2)" - ] - }, - { - "name": "TestHintForIntersectionIndexMerge", - "cases": [ - "set @@tidb_partition_prune_mode = 'dynamic'", - "select * from vh", - "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", - "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", - "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", - "select /*+ qb_name(v, v1@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v1 where c < 30 and d in (2,5)", - "select /*+ use_index_merge(t2, ia, ibc, id, ie) */ * from t2 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", - "select /*+ use_index_merge(t3, ia, ibc, id, ie) */ * from t3 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", - "select /*+ use_index_merge(t4, ia, ibc, id, ie) */ * from t4 where a > 10 and b = 20 and c < 35 and d in (1,3,8,9) and e = 100", - "select /*+ use_index_merge(t5, is1, is2, is3, is4) */ * from t5 where s1 = 'Abc' and s2 > 'zzz' and s3 < 'B啊a' and s4 = 'CcC'", - "select /*+ use_index_merge(t6, primary, is3, is4) */ * from t6 where s1 = 'Abc' and s2 > 'zzz' and s3 = 'A啊a' and s4 not like 'Cd_'", - "select /*+ use_index_merge(t7, primary,ia,ib,ic,ie,iff,ig) */ * from t7 where a = 100 and b > 5 and c < 12.3 and d > 54.321 and e = '2022-11-22 17:00' and f > '2020-6-23 10:00' and g < 2025", - "select /*+ use_index_merge(t8, primary,is2,is3,is4,is5) */ * from t8 where s1 like '啊A%' and s2 > 'abc' and s3 > 'cba' and s4 in ('aA', '??') and s5 = 'test,2'", - "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.c = t2.a) from t2", - "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.c > t2.a) from t2", - "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.e > t2.a) from t2", - "set @@tidb_partition_prune_mode = 'static'", - "select * from vh", - "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", - "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", - "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", - "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", - "select /*+ use_index_merge(t2, ia, ibc, id, ie) */ * from t2 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", - "select /*+ use_index_merge(t3, ia, ibc, id, ie) */ * from t3 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", - "select /*+ use_index_merge(t4, ia, ibc, id, ie) */ * from t4 where a > 10 and b = 20 and c < 35 and d in (1,3,8,9) and e = 100" - ] - } -] diff --git a/planner/core/casetest/testdata/index_merge_suite_out.json b/planner/core/casetest/testdata/index_merge_suite_out.json deleted file mode 100644 index 2c66948aca057..0000000000000 --- a/planner/core/casetest/testdata/index_merge_suite_out.json +++ /dev/null @@ -1,1188 +0,0 @@ -[ - { - "Name": "TestEnforceMVIndex", - "Cases": [ - { - "SQL": "select /*+ use_index(t, kj) */ * from t", - "Plan": null, - "Err": "[planner:1815]Internal : Can't find a proper physical plan for this query" - }, - { - "SQL": "select /*+ use_index(t, kj) */ a from t", - "Plan": null, - "Err": "[planner:1815]Internal : Can't find a proper physical plan for this query" - }, - { - "SQL": "select /*+ use_index(t, kj) */ * from t where a<10", - "Plan": null, - "Err": "[planner:1815]Internal : Can't find a proper physical plan for this query" - }, - { - "SQL": "select /*+ use_index(t, kj) */ * from t where (1 member of (j))", - "Plan": [ - "Selection 8000.00 root json_memberof(cast(1, json BINARY), test.t.j)", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:kj(cast(`j` as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Err": "" - }, - { - "SQL": "select /*+ use_index(t, kj) */ * from t where (1 member of (j)) and a=10", - "Plan": [ - "Selection 8.00 root json_memberof(cast(1, json BINARY), test.t.j)", - "└─IndexMerge 0.01 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:kj(cast(`j` as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 0.01 cop[tikv] eq(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Err": "" - }, - { - "SQL": "select /*+ use_index(t, kj) */ * from t where (1 member of (j)) or a=10", - "Plan": null, - "Err": "[planner:1815]Internal : Can't find a proper physical plan for this query" - }, - { - "SQL": "select /*+ use_index_merge(t, kj) */ * from t", - "Plan": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Err": "" - }, - { - "SQL": "select /*+ use_index_merge(t, kj) */ a from t", - "Plan": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Err": "" - }, - { - "SQL": "select /*+ use_index_merge(t, kj) */ * from t where a<10", - "Plan": [ - "TableReader 3323.33 root data:Selection", - "└─Selection 3323.33 cop[tikv] lt(test.t.a, 10)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Err": "" - }, - { - "SQL": "select /*+ use_index_merge(t, kj) */ * from t where (1 member of (j)) or a=10", - "Plan": [ - "Selection 8000.00 root or(json_memberof(cast(1, json BINARY), test.t.j), eq(test.t.a, 10))", - "└─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - "Err": "" - } - ] - }, - { - "Name": "TestIndexMergeJSONMemberOf", - "Cases": [ - { - "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where (1 member of (j0->'$.path0'))", - "Plan": [ - "Selection 8.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path0\"))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and a<10", - "Plan": [ - "Selection 8.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\"))", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_1(cast(json_extract(`j0`, _utf8mb4'$.path1') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.XXX')) and a<10", - "Plan": [ - "Selection 2658.67 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.XXX\"))", - "└─TableReader 3323.33 root data:Selection", - " └─Selection 3323.33 cop[tikv] lt(test.t.a, 10)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and (2 member of (j1)) and a<10", - "Plan": [ - "Selection 8.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\")), json_memberof(cast(2, json BINARY), test.t.j1)", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_1(cast(json_extract(`j0`, _utf8mb4'$.path1') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index(t, j0_0) */ * from t where (1 member of (j0->'$.path0'))", - "Plan": [ - "Selection 8000.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path0\"))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and a<10", - "Plan": [ - "Selection 2658.67 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\"))", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_1(cast(json_extract(`j0`, _utf8mb4'$.path1') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from t use index(j0_0) where (1 member of (j0->'$.path0'))", - "Plan": [ - "Selection 8000.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path0\"))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from t use index(j0_1) where (1 member of (j0->'$.path1')) and a<10", - "Plan": [ - "Selection 2658.67 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\"))", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_1(cast(json_extract(`j0`, _utf8mb4'$.path1') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from t force index(j0_0) where (1 member of (j0->'$.path0'))", - "Plan": [ - "Selection 8000.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path0\"))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from t force index(j0_1) where (1 member of (j0->'$.path1')) and a<10", - "Plan": [ - "Selection 2658.67 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\"))", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_1(cast(json_extract(`j0`, _utf8mb4'$.path1') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j1) */ * from t where (1 member of (j0->'$.path1')) and (2 member of (j1)) and a<10", - "Plan": [ - "Selection 8.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\")), json_memberof(cast(2, json BINARY), test.t.j1)", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_1(cast(json_extract(`j0`, _utf8mb4'$.path1') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '[1, 2, 3]')", - "Plan": [ - "IndexMerge 10.00 root type: intersection", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[2,2], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[3,3], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '[1, 2, 3]')", - "Plan": [ - "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path0\"), cast(\"[1, 2, 3]\", json BINARY))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[2,2], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[3,3], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('[1, 2, 3]', (j0->'$.path0'))", - "Plan": [ - "Selection 8.00 root json_overlaps(cast(\"[1, 2, 3]\", json BINARY), json_extract(test.t.j0, \"$.path0\"))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[2,2], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[3,3], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '[1, 2, 3]') and a<10", - "Plan": [ - "IndexMerge 3.32 root type: intersection", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[2,2], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[3,3], keep order:false, stats:pseudo", - "└─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '[1, 2, 3]') and a<10", - "Plan": [ - "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path0\"), cast(\"[1, 2, 3]\", json BINARY))", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[2,2], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[3,3], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('[1, 2, 3]', (j0->'$.path0')) and a<10", - "Plan": [ - "Selection 8.00 root json_overlaps(cast(\"[1, 2, 3]\", json BINARY), json_extract(test.t.j0, \"$.path0\"))", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[2,2], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[3,3], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '1')", - "Plan": [ - "IndexMerge 10.00 root type: intersection", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '1')", - "Plan": [ - "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path0\"), cast(\"1\", json BINARY))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('1', (j0->'$.path0'))", - "Plan": [ - "Selection 8.00 root json_overlaps(cast(\"1\", json BINARY), json_extract(test.t.j0, \"$.path0\"))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '1') and a<10", - "Plan": [ - "IndexMerge 3.32 root type: intersection", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - "└─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '1') and a<10", - "Plan": [ - "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path0\"), cast(\"1\", json BINARY))", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('1', (j0->'$.path0')) and a<10", - "Plan": [ - "Selection 8.00 root json_overlaps(cast(\"1\", json BINARY), json_extract(test.t.j0, \"$.path0\"))", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_string) */ * from t where (\"a\" member of (j0->'$.path_string'))", - "Plan": [ - "Selection 8.00 root json_memberof(cast(\"a\", json BINARY), json_extract(test.t.j0, \"$.path_string\"))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x61,0x61], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_string) */ * from t where (\"a\" member of (j0->'$.path_string')) and a<10", - "Plan": [ - "Selection 8.00 root json_memberof(cast(\"a\", json BINARY), json_extract(test.t.j0, \"$.path_string\"))", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x61,0x61], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_string) */ * from t where json_contains((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]')", - "Plan": [ - "IndexMerge 10.00 root type: intersection", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x61,0x61], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x62,0x62], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x63,0x63], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_string) */ * from t where json_contains((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]') and a<10", - "Plan": [ - "IndexMerge 3.32 root type: intersection", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x61,0x61], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x62,0x62], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x63,0x63], keep order:false, stats:pseudo", - "└─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_string) */ * from t where json_overlaps((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]')", - "Plan": [ - "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path_string\"), cast(\"[\"a\", \"b\", \"c\"]\", json BINARY))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x61,0x61], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x62,0x62], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x63,0x63], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_string) */ * from t where json_overlaps((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]') and a<10", - "Plan": [ - "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path_string\"), cast(\"[\"a\", \"b\", \"c\"]\", json BINARY))", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x61,0x61], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x62,0x62], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x63,0x63], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_date) */ * from t where (\"2023-01-01\" member of (j0->'$.path_date'))", - "Plan": [ - "Selection 8.00 root json_memberof(cast(\"2023-01-01\", json BINARY), json_extract(test.t.j0, \"$.path_date\"))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-01,2023-01-01], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_date) */ * from t where (\"2023-01-01\" member of (j0->'$.path_date')) and a<10", - "Plan": [ - "Selection 8.00 root json_memberof(cast(\"2023-01-01\", json BINARY), json_extract(test.t.j0, \"$.path_date\"))", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-01,2023-01-01], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_date) */ * from t where json_contains((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date)))", - "Plan": [ - "IndexMerge 10.00 root type: intersection", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-01,2023-01-01], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-02,2023-01-02], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-03,2023-01-03], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_date) */ * from t where json_contains((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date))) and a<10", - "Plan": [ - "IndexMerge 3.32 root type: intersection", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-01,2023-01-01], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-02,2023-01-02], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-03,2023-01-03], keep order:false, stats:pseudo", - "└─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_date) */ * from t where json_overlaps((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date)))", - "Plan": [ - "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path_date\"), json_array(cast(2023-01-01, json BINARY), cast(2023-01-02, json BINARY), cast(2023-01-03, json BINARY)))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-01,2023-01-01], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-02,2023-01-02], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-03,2023-01-03], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, j0_date) */ * from t where json_overlaps((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date))) and a<10", - "Plan": [ - "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path_date\"), json_array(cast(2023-01-01, json BINARY), cast(2023-01-02, json BINARY), cast(2023-01-03, json BINARY)))", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-01,2023-01-01], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-02,2023-01-02], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-03,2023-01-03], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestCompositeMVIndex", - "Cases": [ - { - "SQL": "select /*+ use_index_merge(t, idx) */ * from t where a=1 and b=2 and (3 member of (j)) and c=4", - "Plan": [ - "Selection 0.00 root json_memberof(cast(3, json BINARY), test.t.j)", - "└─IndexMerge 0.00 root type: union", - " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2 3 4,1 2 3 4], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, idx) */ * from t where a=1 and b=2 and (3 member of (j))", - "Plan": [ - "Selection 0.08 root json_memberof(cast(3, json BINARY), test.t.j)", - "└─IndexMerge 0.00 root type: union", - " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2 3,1 2 3], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, idx) */ * from t where a=1 and b=2", - "Plan": [ - "IndexMerge 0.10 root type: union", - "├─IndexRangeScan(Build) 0.10 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2,1 2], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 0.10 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, idx) */ * from t where a=1", - "Plan": [ - "IndexMerge 10.00 root type: union", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1,1], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, idx2) */ * from t where a=1 and b=2 and ('3' member of (j->'$.str')) and c=4", - "Plan": [ - "Selection 0.00 root json_memberof(cast(\"3\", json BINARY), json_extract(test.t.j, \"$.str\"))", - "└─IndexMerge 0.00 root type: union", - " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(json_extract(`j`, _utf8mb4'$.str') as char(10) array), c) range:[1 2 0x33 4,1 2 0x33 4], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, idx2) */ * from t where a=1 and b=2 and ('3' member of (j->'$.str'))", - "Plan": [ - "Selection 0.08 root json_memberof(cast(\"3\", json BINARY), json_extract(test.t.j, \"$.str\"))", - "└─IndexMerge 0.00 root type: union", - " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(json_extract(`j`, _utf8mb4'$.str') as char(10) array), c) range:[1 2 0x33,1 2 0x33], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, idx2) */ * from t where a=1 and b=2", - "Plan": [ - "IndexMerge 0.10 root type: union", - "├─IndexRangeScan(Build) 0.10 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2,1 2], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 0.10 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, idx2) */ * from t where a=1", - "Plan": [ - "IndexMerge 10.00 root type: union", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1,1], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index(t, idx) */ * from t where a=1 and b=2 and (3 member of (j)) and c=4", - "Plan": [ - "Selection 0.00 root json_memberof(cast(3, json BINARY), test.t.j)", - "└─IndexMerge 0.00 root type: union", - " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2 3 4,1 2 3 4], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from t use index(idx) where a=1 and b=2 and (3 member of (j))", - "Plan": [ - "Selection 0.08 root json_memberof(cast(3, json BINARY), test.t.j)", - "└─IndexMerge 0.00 root type: union", - " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2 3,1 2 3], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index(t, idx) */ * from t where a=1 and b=2", - "Plan": [ - "IndexMerge 0.10 root type: union", - "├─IndexRangeScan(Build) 0.10 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2,1 2], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 0.10 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from t use index(idx) where a=1", - "Plan": [ - "IndexMerge 10.00 root type: union", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1,1], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from t force index(idx) where a=1 and b=2 and (3 member of (j))", - "Plan": [ - "Selection 0.08 root json_memberof(cast(3, json BINARY), test.t.j)", - "└─IndexMerge 0.00 root type: union", - " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2 3,1 2 3], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from t force index(idx) where a=1", - "Plan": [ - "IndexMerge 10.00 root type: union", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1,1], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestDNFOnMVIndex", - "Cases": [ - { - "SQL": "select /*+ use_index_merge(t, idx1) */ * from t where (1 member of (j)) or (2 member of (j))", - "Plan": [ - "Selection 8.00 root or(json_memberof(cast(1, json BINARY), test.t.j), json_memberof(cast(2, json BINARY), test.t.j))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[1,1], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[2,2], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, idx1) */ * from t where ((1 member of (j)) or (2 member of (j))) and (a > 10)", - "Plan": [ - "Selection 8.00 root or(json_memberof(cast(1, json BINARY), test.t.j), json_memberof(cast(2, json BINARY), test.t.j))", - "└─IndexMerge 3.33 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[1,1], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[2,2], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.33 cop[tikv] gt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, idx1) */ * from t where (json_overlaps(j, '[1, 2]')) or (json_overlaps(j, '[3, 4]'))", - "Plan": [ - "Selection 8.00 root or(json_overlaps(test.t.j, cast(\"[1, 2]\", json BINARY)), json_overlaps(test.t.j, cast(\"[3, 4]\", json BINARY)))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[1,1], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[2,2], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[3,3], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[4,4], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, idx1) */ * from t where ((json_overlaps(j, '[1, 2]')) or (json_overlaps(j, '[3, 4]'))) and (a > 10)", - "Plan": [ - "Selection 8.00 root or(json_overlaps(test.t.j, cast(\"[1, 2]\", json BINARY)), json_overlaps(test.t.j, cast(\"[3, 4]\", json BINARY)))", - "└─IndexMerge 3.33 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[1,1], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[2,2], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[3,3], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[4,4], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.33 cop[tikv] gt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, idx1) */ * from t where (json_contains(j, '[1, 2]')) or (json_contains(j, '[3, 4]'))", - "Plan": [ - "TableReader 9600.00 root data:Selection", - "└─Selection 9600.00 cop[tikv] or(json_contains(test.t.j, cast(\"[1, 2]\", json BINARY)), json_contains(test.t.j, cast(\"[3, 4]\", json BINARY)))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, idx2) */ * from t where (a=1 and b=2 and (3 member of (j))) or (a=11 and b=12 and (13 member of (j)))", - "Plan": [ - "Selection 0.00 root or(and(eq(test.t.a, 1), and(eq(test.t.b, 2), json_memberof(cast(3, json BINARY), test.t.j))), and(eq(test.t.a, 11), and(eq(test.t.b, 12), json_memberof(cast(13, json BINARY), test.t.j))))", - "└─IndexMerge 0.00 root type: union", - " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(`j` as signed array), c) range:[1 2 3,1 2 3], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(`j` as signed array), c) range:[11 12 13,11 12 13], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, idx2) */ * from t where (a=1 and b=2 and (3 member of (j))) or (a=11 and b=12 and (13 member of (j)) and c=14)", - "Plan": [ - "Selection 0.00 root or(and(eq(test.t.a, 1), and(eq(test.t.b, 2), json_memberof(cast(3, json BINARY), test.t.j))), and(and(eq(test.t.a, 11), eq(test.t.b, 12)), and(json_memberof(cast(13, json BINARY), test.t.j), eq(test.t.c, 14))))", - "└─IndexMerge 0.00 root type: union", - " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(`j` as signed array), c) range:[1 2 3,1 2 3], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(`j` as signed array), c) range:[11 12 13 14,11 12 13 14], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select /*+ use_index_merge(t, idx2) */ * from t where ((a=1 and b=2 and (3 member of (j))) or (a=11 and b=12 and (13 member of (j)))) and (c > 10)", - "Plan": [ - "Selection 0.00 root or(and(eq(test.t.a, 1), and(eq(test.t.b, 2), json_memberof(cast(3, json BINARY), test.t.j))), and(eq(test.t.a, 11), and(eq(test.t.b, 12), json_memberof(cast(13, json BINARY), test.t.j))))", - "└─IndexMerge 0.00 root type: union", - " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(`j` as signed array), c) range:[1 2 3,1 2 3], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(`j` as signed array), c) range:[11 12 13,11 12 13], keep order:false, stats:pseudo", - " └─Selection(Probe) 0.00 cop[tikv] gt(test.t.c, 10)", - " └─TableRowIDScan 0.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestMVIndexSelection", - "Cases": [ - { - "SQL": "select (j->'$.int') from t where (1 member of (j->'$.int'))", - "Plan": [ - "Projection 8000.00 root json_extract(test.t.j, $.int)->Column#5", - "└─Selection 8000.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j, \"$.int\"))", - " └─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from t where (1 member of (j->'$.int'))", - "Plan": [ - "Selection 8000.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j, \"$.int\"))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from t where (1 member of (j->'$.int')) and a<10", - "Plan": [ - "Selection 2658.67 root json_memberof(cast(1, json BINARY), json_extract(test.t.j, \"$.int\"))", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select (j->'$.int') from t where json_contains((j->'$.int'), '[1, 2, 3]')", - "Plan": [ - "Projection 8000.00 root json_extract(test.t.j, $.int)->Column#5", - "└─IndexMerge 10.00 root type: intersection", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[2,2], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[3,3], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from t where json_contains((j->'$.int'), '[1, 2, 3]')", - "Plan": [ - "IndexMerge 10.00 root type: intersection", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[2,2], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[3,3], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from t where json_contains((j->'$.int'), '[1, 2, 3]') and a<10", - "Plan": [ - "IndexMerge 3.32 root type: intersection", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[2,2], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[3,3], keep order:false, stats:pseudo", - "└─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select (j->'$.int') from t where json_overlaps((j->'$.int'), '[1, 2, 3]')", - "Plan": [ - "Projection 8000.00 root json_extract(test.t.j, $.int)->Column#5", - "└─Selection 8000.00 root json_overlaps(json_extract(test.t.j, \"$.int\"), cast(\"[1, 2, 3]\", json BINARY))", - " └─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[2,2], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[3,3], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from t where json_overlaps((j->'$.int'), '[1, 2, 3]')", - "Plan": [ - "Selection 8000.00 root json_overlaps(json_extract(test.t.j, \"$.int\"), cast(\"[1, 2, 3]\", json BINARY))", - "└─IndexMerge 10.00 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[2,2], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[3,3], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "select * from t where json_overlaps((j->'$.int'), '[1, 2, 3]') and a<10", - "Plan": [ - "Selection 2658.67 root json_overlaps(json_extract(test.t.j, \"$.int\"), cast(\"[1, 2, 3]\", json BINARY))", - "└─IndexMerge 3.32 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[2,2], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[3,3], keep order:false, stats:pseudo", - " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", - " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestIndexMergePathGeneration", - "Cases": [ - "[]", - "[]", - "[{Idxs:[c_d_e,f],TbFilters:[]}]", - "[{Idxs:[c_d_e,f],TbFilters:[or(gt(test.t.c, 5), lt(test.t.f, 7))]},{Idxs:[c_d_e,f],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2))]}]", - "[{Idxs:[c_d_e,f],TbFilters:[or(gt(test.t.c, 5), lt(test.t.f, 7)),or(lt(test.t.c, 1), gt(test.t.g, 2))]},{Idxs:[c_d_e,f],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2)),or(lt(test.t.c, 1), gt(test.t.g, 2))]},{Idxs:[c_d_e,g],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2)),or(gt(test.t.c, 5), lt(test.t.f, 7))]}]", - "[{Idxs:[c_d_e,f],TbFilters:[or(gt(test.t.c, 5), lt(test.t.f, 7)),or(lt(test.t.e, 1), gt(test.t.f, 2))]},{Idxs:[c_d_e,f],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2)),or(lt(test.t.e, 1), gt(test.t.f, 2))]}]" - ] - }, - { - "Name": "TestHintForIntersectionIndexMerge", - "Cases": [ - { - "SQL": "set @@tidb_partition_prune_mode = 'dynamic'", - "Plan": null, - "Result": null - }, - { - "SQL": "select * from vh", - "Plan": [ - "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", - "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", - "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" - ], - "Result": [ - "10 20 5 5 3" - ] - }, - { - "SQL": "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", - "Plan": [ - "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", - "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", - "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" - ], - "Result": [ - "10 20 5 5 3" - ] - }, - { - "SQL": "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", - "Plan": [ - "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", - "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", - "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" - ], - "Result": [ - "10 20 5 5 3" - ] - }, - { - "SQL": "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", - "Plan": [ - "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", - "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", - "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" - ], - "Result": [ - "10 20 5 5 3" - ] - }, - { - "SQL": "select /*+ qb_name(v, v1@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v1 where c < 30 and d in (2,5)", - "Plan": [ - "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", - "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", - "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" - ], - "Result": [ - "10 20 5 5 3" - ] - }, - { - "SQL": "select /*+ use_index_merge(t2, ia, ibc, id, ie) */ * from t2 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", - "Plan": [ - "IndexMerge 0.89 root partition:p0,p1,p2 type: intersection", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:ia(a) range:(10,+inf], keep order:false", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", - "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t2, index:id(d) range:[-inf,45), keep order:false", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:ie(e) range:[100,100], keep order:false", - "└─TableRowIDScan(Probe) 0.89 cop[tikv] table:t2 keep order:false" - ], - "Result": [ - "20 20 20 5 100" - ] - }, - { - "SQL": "select /*+ use_index_merge(t3, ia, ibc, id, ie) */ * from t3 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", - "Plan": [ - "IndexMerge 0.44 root partition:p0 type: intersection", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t3, index:ia(a) range:(10,+inf], keep order:false", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", - "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t3, index:id(d) range:[-inf,45), keep order:false", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, index:ie(e) range:[100,100], keep order:false", - "└─TableRowIDScan(Probe) 0.44 cop[tikv] table:t3 keep order:false" - ], - "Result": [ - "20 20 20 5 100" - ] - }, - { - "SQL": "select /*+ use_index_merge(t4, ia, ibc, id, ie) */ * from t4 where a > 10 and b = 20 and c < 35 and d in (1,3,8,9) and e = 100", - "Plan": [ - "IndexMerge 0.89 root partition:p0,p1 type: intersection", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, index:ia(a) range:(10,+inf], keep order:false", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", - "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t4, index:id(d) range:[1,1], [3,3], [8,8], [9,9], keep order:false", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, index:ie(e) range:[100,100], keep order:false", - "└─TableRowIDScan(Probe) 0.89 cop[tikv] table:t4 keep order:false" - ], - "Result": [ - "30 20 5 8 100" - ] - }, - { - "SQL": "select /*+ use_index_merge(t5, is1, is2, is3, is4) */ * from t5 where s1 = 'Abc' and s2 > 'zzz' and s3 < 'B啊a' and s4 = 'CcC'", - "Plan": [ - "IndexMerge 0.00 root type: intersection", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t5, index:is1(s1) range:[\"Abc\",\"Abc\"], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t5, index:is2(s2) range:(\"zzz\",+inf], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t5, index:is3(s3) range:[-inf,\"B啊a\"), keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t5, index:is4(s4) range:[\"CcC\",\"CcC\"], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 0.00 cop[tikv] table:t5 keep order:false, stats:pseudo" - ], - "Result": [ - "Abc zzzz aa ccc" - ] - }, - { - "SQL": "select /*+ use_index_merge(t6, primary, is3, is4) */ * from t6 where s1 = 'Abc' and s2 > 'zzz' and s3 = 'A啊a' and s4 not like 'Cd_'", - "Plan": [ - "IndexMerge 0.03 root type: intersection", - "├─IndexRangeScan(Build) 33.33 cop[tikv] table:t6, index:PRIMARY(s1, s2) range:(\"Abc\" \"zzz\",\"Abc\" +inf], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t6, index:is3(s3) range:[\"A啊a\",\"A啊a\"], keep order:false, stats:pseudo", - "└─Selection(Probe) 0.03 cop[tikv] gt(test.t6.s2, \"zzz\"), not(like(test.t6.s4, \"Cd_\", 92))", - " └─TableRowIDScan 0.03 cop[tikv] table:t6 keep order:false, stats:pseudo" - ], - "Result": [ - "Abc zzzz A啊A Cdaa" - ] - }, - { - "SQL": "select /*+ use_index_merge(t7, primary,ia,ib,ic,ie,iff,ig) */ * from t7 where a = 100 and b > 5 and c < 12.3 and d > 54.321 and e = '2022-11-22 17:00' and f > '2020-6-23 10:00' and g < 2025", - "Plan": [ - "IndexMerge 0.00 root type: intersection", - "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t7, index:PRIMARY(d) range:(54.321,+inf], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t7, index:ia(a) range:[100,100], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t7, index:ib(b) range:(\"0x05\",+inf], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t7, index:ic(c) range:[-inf,12.3), keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t7, index:ie(e) range:[2022-11-22 17:00:00,2022-11-22 17:00:00], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t7, index:iff(f) range:(2020-06-23 10:00:00.00000,+inf], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t7, index:ig(g) range:[-inf,2025), keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 0.00 cop[tikv] table:t7 keep order:false, stats:pseudo" - ], - "Result": [ - "100 \u0006 12.2 56.000 2022-11-22 17:00:00 2022-12-21 00:00:00.00000 2021" - ] - }, - { - "SQL": "select /*+ use_index_merge(t8, primary,is2,is3,is4,is5) */ * from t8 where s1 like '啊A%' and s2 > 'abc' and s3 > 'cba' and s4 in ('aA', '??') and s5 = 'test,2'", - "Plan": [ - "Selection 1.42 root eq(test.t8.s5, \"test,2\")", - "└─IndexMerge 0.59 root type: intersection", - " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t8, index:is2(s2) range:(0x616263,+inf], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t8, index:is3(s3) range:(0x636261,+inf], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t8, index:is4(s4) range:[\"aA\",\"aA\"], [\"??\",\"??\"], keep order:false, stats:pseudo", - " └─Selection(Probe) 0.59 cop[tikv] gt(test.t8.s3, \"cba\"), like(test.t8.s1, \"啊A%\", 92)", - " └─TableRowIDScan 2.22 cop[tikv] table:t8 keep order:false, stats:pseudo" - ], - "Result": [ - "啊aabbccdd abcc cccc aA tEsT,2" - ] - }, - { - "SQL": "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.c = t2.a) from t2", - "Plan": [ - "Projection 3.00 root test.t1.a", - "└─Apply 3.00 root CARTESIAN left outer join", - " ├─IndexReader(Build) 3.00 root partition:all index:IndexFullScan", - " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:ia(a) keep order:false", - " └─MaxOneRow(Probe) 3.00 root ", - " └─IndexMerge 1.50 root partition:all type: intersection", - " ├─IndexRangeScan(Build) 6.00 cop[tikv] table:t1, index:ia(a) range:(10,+inf], keep order:false", - " ├─IndexRangeScan(Build) 4.00 cop[tikv] table:t1, index:ibc(b, c) range: decided by [eq(test.t1.b, 20) eq(test.t1.c, test.t2.a)], keep order:false", - " └─TableRowIDScan(Probe) 1.50 cop[tikv] table:t1 keep order:false" - ], - "Result": [ - "20", - "", - "" - ] - }, - { - "SQL": "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.c > t2.a) from t2", - "Plan": [ - "Projection 3.00 root test.t1.a", - "└─Apply 3.00 root CARTESIAN left outer join", - " ├─IndexReader(Build) 3.00 root partition:all index:IndexFullScan", - " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:ia(a) keep order:false", - " └─MaxOneRow(Probe) 3.00 root ", - " └─IndexMerge 3.60 root partition:all type: intersection", - " ├─IndexRangeScan(Build) 6.00 cop[tikv] table:t1, index:ia(a) range:(10,+inf], keep order:false", - " ├─Selection(Build) 7.20 cop[tikv] gt(test.t1.c, test.t2.a)", - " │ └─IndexRangeScan 9.00 cop[tikv] table:t1, index:ibc(b, c) range:[20,20], keep order:false", - " └─TableRowIDScan(Probe) 3.60 cop[tikv] table:t1 keep order:false" - ], - "Result": [ - "20", - "20", - "" - ] - }, - { - "SQL": "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.e > t2.a) from t2", - "Plan": [ - "Projection 3.00 root test.t1.a", - "└─Apply 3.00 root CARTESIAN left outer join", - " ├─IndexReader(Build) 3.00 root partition:all index:IndexFullScan", - " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:ia(a) keep order:false", - " └─MaxOneRow(Probe) 3.00 root ", - " └─IndexMerge 3.60 root partition:all type: intersection", - " ├─IndexRangeScan(Build) 6.00 cop[tikv] table:t1, index:ia(a) range:(10,+inf], keep order:false", - " ├─IndexRangeScan(Build) 9.00 cop[tikv] table:t1, index:ibc(b, c) range:[20,20], keep order:false", - " └─Selection(Probe) 3.60 cop[tikv] gt(test.t1.e, test.t2.a)", - " └─TableRowIDScan 4.50 cop[tikv] table:t1 keep order:false" - ], - "Result": [ - "20", - "20", - "20" - ] - }, - { - "SQL": "set @@tidb_partition_prune_mode = 'static'", - "Plan": null, - "Result": null - }, - { - "SQL": "select * from vh", - "Plan": [ - "PartitionUnion 0.50 root ", - "├─IndexMerge 0.50 root type: intersection", - "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", - "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", - "├─IndexMerge 1.00 root type: intersection", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", - "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", - "└─IndexMerge 0.00 root type: intersection", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" - ], - "Result": [ - "10 20 5 5 3" - ] - }, - { - "SQL": "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", - "Plan": [ - "PartitionUnion 0.50 root ", - "├─IndexMerge 0.50 root type: intersection", - "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", - "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", - "├─IndexMerge 1.00 root type: intersection", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", - "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", - "└─IndexMerge 0.00 root type: intersection", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" - ], - "Result": [ - "10 20 5 5 3" - ] - }, - { - "SQL": "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", - "Plan": [ - "PartitionUnion 0.50 root ", - "├─IndexMerge 0.50 root type: intersection", - "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", - "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", - "├─IndexMerge 1.00 root type: intersection", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", - "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", - "└─IndexMerge 0.00 root type: intersection", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" - ], - "Result": [ - "10 20 5 5 3" - ] - }, - { - "SQL": "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", - "Plan": [ - "PartitionUnion 0.50 root ", - "├─IndexMerge 0.50 root type: intersection", - "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", - "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", - "├─IndexMerge 1.00 root type: intersection", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", - "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", - "└─IndexMerge 0.00 root type: intersection", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" - ], - "Result": [ - "10 20 5 5 3" - ] - }, - { - "SQL": "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", - "Plan": [ - "PartitionUnion 0.50 root ", - "├─IndexMerge 0.50 root type: intersection", - "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", - "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", - "├─IndexMerge 1.00 root type: intersection", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", - "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", - "└─IndexMerge 0.00 root type: intersection", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" - ], - "Result": [ - "10 20 5 5 3" - ] - }, - { - "SQL": "select /*+ use_index_merge(t2, ia, ibc, id, ie) */ * from t2 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", - "Plan": [ - "PartitionUnion 1.25 root ", - "├─IndexMerge 0.25 root type: intersection", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p0, index:ia(a) range:(10,+inf], keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p0, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", - "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, partition:p0, index:id(d) range:[-inf,45), keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p0, index:ie(e) range:[100,100], keep order:false", - "│ └─TableRowIDScan(Probe) 0.25 cop[tikv] table:t2, partition:p0 keep order:false", - "├─IndexMerge 1.00 root type: intersection", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:ia(a) range:(10,+inf], keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:id(d) range:[-inf,45), keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:ie(e) range:[100,100], keep order:false", - "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2, partition:p1 keep order:false", - "└─IndexMerge 0.00 root type: intersection", - " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t2, partition:p2, index:ia(a) range:(10,+inf], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t2, partition:p2, index:ibc(b, c) range:[20 -inf,20 35), keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t2, partition:p2, index:id(d) range:[-inf,45), keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, partition:p2, index:ie(e) range:[100,100], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo" - ], - "Result": [ - "20 20 20 5 100" - ] - }, - { - "SQL": "select /*+ use_index_merge(t3, ia, ibc, id, ie) */ * from t3 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", - "Plan": [ - "IndexMerge 0.50 root type: intersection", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t3, partition:p0, index:ia(a) range:(10,+inf], keep order:false", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t3, partition:p0, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, partition:p0, index:id(d) range:[-inf,45), keep order:false", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, partition:p0, index:ie(e) range:[100,100], keep order:false", - "└─TableRowIDScan(Probe) 0.50 cop[tikv] table:t3, partition:p0 keep order:false" - ], - "Result": [ - "20 20 20 5 100" - ] - }, - { - "SQL": "select /*+ use_index_merge(t4, ia, ibc, id, ie) */ * from t4 where a > 10 and b = 20 and c < 35 and d in (1,3,8,9) and e = 100", - "Plan": [ - "PartitionUnion 1.25 root ", - "├─IndexMerge 0.25 root type: intersection", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p0, index:ia(a) range:(10,+inf], keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p0, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", - "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, partition:p0, index:id(d) range:[1,1], [3,3], [8,8], [9,9], keep order:false", - "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p0, index:ie(e) range:[100,100], keep order:false", - "│ └─TableRowIDScan(Probe) 0.25 cop[tikv] table:t4, partition:p0 keep order:false", - "└─IndexMerge 1.00 root type: intersection", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:ia(a) range:(10,+inf], keep order:false", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:id(d) range:[1,1], [3,3], [8,8], [9,9], keep order:false", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:ie(e) range:[100,100], keep order:false", - " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t4, partition:p1 keep order:false" - ], - "Result": [ - "30 20 5 8 100" - ] - } - ] - } -] From a77457570e409e8d8e75466f2c7c4e0180f4de24 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 15 Feb 2023 12:04:32 +0800 Subject: [PATCH 11/12] *: bazel upgrade shard_count Signed-off-by: Weizhen Wang --- .../core/testdata/index_merge_suite_in.json | 144 ++ .../core/testdata/index_merge_suite_out.json | 1188 +++++++++++++++++ 2 files changed, 1332 insertions(+) create mode 100644 planner/core/testdata/index_merge_suite_in.json create mode 100644 planner/core/testdata/index_merge_suite_out.json diff --git a/planner/core/testdata/index_merge_suite_in.json b/planner/core/testdata/index_merge_suite_in.json new file mode 100644 index 0000000000000..d660364305397 --- /dev/null +++ b/planner/core/testdata/index_merge_suite_in.json @@ -0,0 +1,144 @@ +[ + { + "name": "TestEnforceMVIndex", + "cases": [ + "select /*+ use_index(t, kj) */ * from t", + "select /*+ use_index(t, kj) */ a from t", + "select /*+ use_index(t, kj) */ * from t where a<10", + "select /*+ use_index(t, kj) */ * from t where (1 member of (j))", + "select /*+ use_index(t, kj) */ * from t where (1 member of (j)) and a=10", + "select /*+ use_index(t, kj) */ * from t where (1 member of (j)) or a=10", + "select /*+ use_index_merge(t, kj) */ * from t", + "select /*+ use_index_merge(t, kj) */ a from t", + "select /*+ use_index_merge(t, kj) */ * from t where a<10", + "select /*+ use_index_merge(t, kj) */ * from t where (1 member of (j)) or a=10" + ] + }, + { + "name": "TestIndexMergeJSONMemberOf", + "cases": [ + "select /*+ use_index_merge(t, j0_0) */ * from t where (1 member of (j0->'$.path0'))", + "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and a<10", + "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.XXX')) and a<10", + "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and (2 member of (j1)) and a<10", + "select /*+ use_index(t, j0_0) */ * from t where (1 member of (j0->'$.path0'))", + "select /*+ use_index(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and a<10", + "select * from t use index(j0_0) where (1 member of (j0->'$.path0'))", + "select * from t use index(j0_1) where (1 member of (j0->'$.path1')) and a<10", + "select * from t force index(j0_0) where (1 member of (j0->'$.path0'))", + "select * from t force index(j0_1) where (1 member of (j0->'$.path1')) and a<10", + "select /*+ use_index_merge(t, j1) */ * from t where (1 member of (j0->'$.path1')) and (2 member of (j1)) and a<10", + "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '[1, 2, 3]')", + "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '[1, 2, 3]')", + "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('[1, 2, 3]', (j0->'$.path0'))", + "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '[1, 2, 3]') and a<10", + "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '[1, 2, 3]') and a<10", + "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('[1, 2, 3]', (j0->'$.path0')) and a<10", + "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '1')", + "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '1')", + "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('1', (j0->'$.path0'))", + "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '1') and a<10", + "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '1') and a<10", + "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('1', (j0->'$.path0')) and a<10", + "select /*+ use_index_merge(t, j0_string) */ * from t where (\"a\" member of (j0->'$.path_string'))", + "select /*+ use_index_merge(t, j0_string) */ * from t where (\"a\" member of (j0->'$.path_string')) and a<10", + "select /*+ use_index_merge(t, j0_string) */ * from t where json_contains((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]')", + "select /*+ use_index_merge(t, j0_string) */ * from t where json_contains((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]') and a<10", + "select /*+ use_index_merge(t, j0_string) */ * from t where json_overlaps((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]')", + "select /*+ use_index_merge(t, j0_string) */ * from t where json_overlaps((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]') and a<10", + "select /*+ use_index_merge(t, j0_date) */ * from t where (\"2023-01-01\" member of (j0->'$.path_date'))", + "select /*+ use_index_merge(t, j0_date) */ * from t where (\"2023-01-01\" member of (j0->'$.path_date')) and a<10", + "select /*+ use_index_merge(t, j0_date) */ * from t where json_contains((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date)))", + "select /*+ use_index_merge(t, j0_date) */ * from t where json_contains((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date))) and a<10", + "select /*+ use_index_merge(t, j0_date) */ * from t where json_overlaps((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date)))", + "select /*+ use_index_merge(t, j0_date) */ * from t where json_overlaps((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date))) and a<10" + ] + }, + { + "name": "TestCompositeMVIndex", + "cases": [ + "select /*+ use_index_merge(t, idx) */ * from t where a=1 and b=2 and (3 member of (j)) and c=4", + "select /*+ use_index_merge(t, idx) */ * from t where a=1 and b=2 and (3 member of (j))", + "select /*+ use_index_merge(t, idx) */ * from t where a=1 and b=2", + "select /*+ use_index_merge(t, idx) */ * from t where a=1", + "select /*+ use_index_merge(t, idx2) */ * from t where a=1 and b=2 and ('3' member of (j->'$.str')) and c=4", + "select /*+ use_index_merge(t, idx2) */ * from t where a=1 and b=2 and ('3' member of (j->'$.str'))", + "select /*+ use_index_merge(t, idx2) */ * from t where a=1 and b=2", + "select /*+ use_index_merge(t, idx2) */ * from t where a=1", + "select /*+ use_index(t, idx) */ * from t where a=1 and b=2 and (3 member of (j)) and c=4", + "select * from t use index(idx) where a=1 and b=2 and (3 member of (j))", + "select /*+ use_index(t, idx) */ * from t where a=1 and b=2", + "select * from t use index(idx) where a=1", + "select * from t force index(idx) where a=1 and b=2 and (3 member of (j))", + "select * from t force index(idx) where a=1" + ] + }, + { + "name": "TestDNFOnMVIndex", + "cases": [ + "select /*+ use_index_merge(t, idx1) */ * from t where (1 member of (j)) or (2 member of (j))", + "select /*+ use_index_merge(t, idx1) */ * from t where ((1 member of (j)) or (2 member of (j))) and (a > 10)", + "select /*+ use_index_merge(t, idx1) */ * from t where (json_overlaps(j, '[1, 2]')) or (json_overlaps(j, '[3, 4]'))", + "select /*+ use_index_merge(t, idx1) */ * from t where ((json_overlaps(j, '[1, 2]')) or (json_overlaps(j, '[3, 4]'))) and (a > 10)", + "select /*+ use_index_merge(t, idx1) */ * from t where (json_contains(j, '[1, 2]')) or (json_contains(j, '[3, 4]'))", + "select /*+ use_index_merge(t, idx2) */ * from t where (a=1 and b=2 and (3 member of (j))) or (a=11 and b=12 and (13 member of (j)))", + "select /*+ use_index_merge(t, idx2) */ * from t where (a=1 and b=2 and (3 member of (j))) or (a=11 and b=12 and (13 member of (j)) and c=14)", + "select /*+ use_index_merge(t, idx2) */ * from t where ((a=1 and b=2 and (3 member of (j))) or (a=11 and b=12 and (13 member of (j)))) and (c > 10)" + ] + }, + { + "name": "TestMVIndexSelection", + "cases": [ + "select (j->'$.int') from t where (1 member of (j->'$.int'))", + "select * from t where (1 member of (j->'$.int'))", + "select * from t where (1 member of (j->'$.int')) and a<10", + "select (j->'$.int') from t where json_contains((j->'$.int'), '[1, 2, 3]')", + "select * from t where json_contains((j->'$.int'), '[1, 2, 3]')", + "select * from t where json_contains((j->'$.int'), '[1, 2, 3]') and a<10", + "select (j->'$.int') from t where json_overlaps((j->'$.int'), '[1, 2, 3]')", + "select * from t where json_overlaps((j->'$.int'), '[1, 2, 3]')", + "select * from t where json_overlaps((j->'$.int'), '[1, 2, 3]') and a<10" + ] + }, + { + "name": "TestIndexMergePathGeneration", + "cases": [ + "select * from t", + "select * from t where c < 1", + "select * from t where c < 1 or f > 2", + "select * from t where (c < 1 or f > 2) and (c > 5 or f < 7)", + "select * from t where (c < 1 or f > 2) and (c > 5 or f < 7) and (c < 1 or g > 2)", + "select * from t where (c < 1 or f > 2) and (c > 5 or f < 7) and (e < 1 or f > 2)" + ] + }, + { + "name": "TestHintForIntersectionIndexMerge", + "cases": [ + "set @@tidb_partition_prune_mode = 'dynamic'", + "select * from vh", + "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "select /*+ qb_name(v, v1@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v1 where c < 30 and d in (2,5)", + "select /*+ use_index_merge(t2, ia, ibc, id, ie) */ * from t2 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "select /*+ use_index_merge(t3, ia, ibc, id, ie) */ * from t3 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "select /*+ use_index_merge(t4, ia, ibc, id, ie) */ * from t4 where a > 10 and b = 20 and c < 35 and d in (1,3,8,9) and e = 100", + "select /*+ use_index_merge(t5, is1, is2, is3, is4) */ * from t5 where s1 = 'Abc' and s2 > 'zzz' and s3 < 'B啊a' and s4 = 'CcC'", + "select /*+ use_index_merge(t6, primary, is3, is4) */ * from t6 where s1 = 'Abc' and s2 > 'zzz' and s3 = 'A啊a' and s4 not like 'Cd_'", + "select /*+ use_index_merge(t7, primary,ia,ib,ic,ie,iff,ig) */ * from t7 where a = 100 and b > 5 and c < 12.3 and d > 54.321 and e = '2022-11-22 17:00' and f > '2020-6-23 10:00' and g < 2025", + "select /*+ use_index_merge(t8, primary,is2,is3,is4,is5) */ * from t8 where s1 like '啊A%' and s2 > 'abc' and s3 > 'cba' and s4 in ('aA', '??') and s5 = 'test,2'", + "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.c = t2.a) from t2", + "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.c > t2.a) from t2", + "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.e > t2.a) from t2", + "set @@tidb_partition_prune_mode = 'static'", + "select * from vh", + "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "select /*+ use_index_merge(t2, ia, ibc, id, ie) */ * from t2 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "select /*+ use_index_merge(t3, ia, ibc, id, ie) */ * from t3 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "select /*+ use_index_merge(t4, ia, ibc, id, ie) */ * from t4 where a > 10 and b = 20 and c < 35 and d in (1,3,8,9) and e = 100" + ] + } +] diff --git a/planner/core/testdata/index_merge_suite_out.json b/planner/core/testdata/index_merge_suite_out.json new file mode 100644 index 0000000000000..2c66948aca057 --- /dev/null +++ b/planner/core/testdata/index_merge_suite_out.json @@ -0,0 +1,1188 @@ +[ + { + "Name": "TestEnforceMVIndex", + "Cases": [ + { + "SQL": "select /*+ use_index(t, kj) */ * from t", + "Plan": null, + "Err": "[planner:1815]Internal : Can't find a proper physical plan for this query" + }, + { + "SQL": "select /*+ use_index(t, kj) */ a from t", + "Plan": null, + "Err": "[planner:1815]Internal : Can't find a proper physical plan for this query" + }, + { + "SQL": "select /*+ use_index(t, kj) */ * from t where a<10", + "Plan": null, + "Err": "[planner:1815]Internal : Can't find a proper physical plan for this query" + }, + { + "SQL": "select /*+ use_index(t, kj) */ * from t where (1 member of (j))", + "Plan": [ + "Selection 8000.00 root json_memberof(cast(1, json BINARY), test.t.j)", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:kj(cast(`j` as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Err": "" + }, + { + "SQL": "select /*+ use_index(t, kj) */ * from t where (1 member of (j)) and a=10", + "Plan": [ + "Selection 8.00 root json_memberof(cast(1, json BINARY), test.t.j)", + "└─IndexMerge 0.01 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:kj(cast(`j` as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.01 cop[tikv] eq(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Err": "" + }, + { + "SQL": "select /*+ use_index(t, kj) */ * from t where (1 member of (j)) or a=10", + "Plan": null, + "Err": "[planner:1815]Internal : Can't find a proper physical plan for this query" + }, + { + "SQL": "select /*+ use_index_merge(t, kj) */ * from t", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Err": "" + }, + { + "SQL": "select /*+ use_index_merge(t, kj) */ a from t", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Err": "" + }, + { + "SQL": "select /*+ use_index_merge(t, kj) */ * from t where a<10", + "Plan": [ + "TableReader 3323.33 root data:Selection", + "└─Selection 3323.33 cop[tikv] lt(test.t.a, 10)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Err": "" + }, + { + "SQL": "select /*+ use_index_merge(t, kj) */ * from t where (1 member of (j)) or a=10", + "Plan": [ + "Selection 8000.00 root or(json_memberof(cast(1, json BINARY), test.t.j), eq(test.t.a, 10))", + "└─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Err": "" + } + ] + }, + { + "Name": "TestIndexMergeJSONMemberOf", + "Cases": [ + { + "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where (1 member of (j0->'$.path0'))", + "Plan": [ + "Selection 8.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path0\"))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and a<10", + "Plan": [ + "Selection 8.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\"))", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_1(cast(json_extract(`j0`, _utf8mb4'$.path1') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.XXX')) and a<10", + "Plan": [ + "Selection 2658.67 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.XXX\"))", + "└─TableReader 3323.33 root data:Selection", + " └─Selection 3323.33 cop[tikv] lt(test.t.a, 10)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and (2 member of (j1)) and a<10", + "Plan": [ + "Selection 8.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\")), json_memberof(cast(2, json BINARY), test.t.j1)", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_1(cast(json_extract(`j0`, _utf8mb4'$.path1') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index(t, j0_0) */ * from t where (1 member of (j0->'$.path0'))", + "Plan": [ + "Selection 8000.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path0\"))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and a<10", + "Plan": [ + "Selection 2658.67 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\"))", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_1(cast(json_extract(`j0`, _utf8mb4'$.path1') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from t use index(j0_0) where (1 member of (j0->'$.path0'))", + "Plan": [ + "Selection 8000.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path0\"))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from t use index(j0_1) where (1 member of (j0->'$.path1')) and a<10", + "Plan": [ + "Selection 2658.67 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\"))", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_1(cast(json_extract(`j0`, _utf8mb4'$.path1') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from t force index(j0_0) where (1 member of (j0->'$.path0'))", + "Plan": [ + "Selection 8000.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path0\"))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from t force index(j0_1) where (1 member of (j0->'$.path1')) and a<10", + "Plan": [ + "Selection 2658.67 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\"))", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_1(cast(json_extract(`j0`, _utf8mb4'$.path1') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j1) */ * from t where (1 member of (j0->'$.path1')) and (2 member of (j1)) and a<10", + "Plan": [ + "Selection 8.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\")), json_memberof(cast(2, json BINARY), test.t.j1)", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_1(cast(json_extract(`j0`, _utf8mb4'$.path1') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '[1, 2, 3]')", + "Plan": [ + "IndexMerge 10.00 root type: intersection", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[2,2], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[3,3], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '[1, 2, 3]')", + "Plan": [ + "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path0\"), cast(\"[1, 2, 3]\", json BINARY))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[2,2], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[3,3], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('[1, 2, 3]', (j0->'$.path0'))", + "Plan": [ + "Selection 8.00 root json_overlaps(cast(\"[1, 2, 3]\", json BINARY), json_extract(test.t.j0, \"$.path0\"))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[2,2], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[3,3], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '[1, 2, 3]') and a<10", + "Plan": [ + "IndexMerge 3.32 root type: intersection", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[2,2], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[3,3], keep order:false, stats:pseudo", + "└─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '[1, 2, 3]') and a<10", + "Plan": [ + "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path0\"), cast(\"[1, 2, 3]\", json BINARY))", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[2,2], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[3,3], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('[1, 2, 3]', (j0->'$.path0')) and a<10", + "Plan": [ + "Selection 8.00 root json_overlaps(cast(\"[1, 2, 3]\", json BINARY), json_extract(test.t.j0, \"$.path0\"))", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[2,2], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[3,3], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '1')", + "Plan": [ + "IndexMerge 10.00 root type: intersection", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '1')", + "Plan": [ + "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path0\"), cast(\"1\", json BINARY))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('1', (j0->'$.path0'))", + "Plan": [ + "Selection 8.00 root json_overlaps(cast(\"1\", json BINARY), json_extract(test.t.j0, \"$.path0\"))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_contains((j0->'$.path0'), '1') and a<10", + "Plan": [ + "IndexMerge 3.32 root type: intersection", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + "└─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps((j0->'$.path0'), '1') and a<10", + "Plan": [ + "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path0\"), cast(\"1\", json BINARY))", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where json_overlaps('1', (j0->'$.path0')) and a<10", + "Plan": [ + "Selection 8.00 root json_overlaps(cast(\"1\", json BINARY), json_extract(test.t.j0, \"$.path0\"))", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_string) */ * from t where (\"a\" member of (j0->'$.path_string'))", + "Plan": [ + "Selection 8.00 root json_memberof(cast(\"a\", json BINARY), json_extract(test.t.j0, \"$.path_string\"))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x61,0x61], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_string) */ * from t where (\"a\" member of (j0->'$.path_string')) and a<10", + "Plan": [ + "Selection 8.00 root json_memberof(cast(\"a\", json BINARY), json_extract(test.t.j0, \"$.path_string\"))", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x61,0x61], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_string) */ * from t where json_contains((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]')", + "Plan": [ + "IndexMerge 10.00 root type: intersection", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x61,0x61], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x62,0x62], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x63,0x63], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_string) */ * from t where json_contains((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]') and a<10", + "Plan": [ + "IndexMerge 3.32 root type: intersection", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x61,0x61], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x62,0x62], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x63,0x63], keep order:false, stats:pseudo", + "└─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_string) */ * from t where json_overlaps((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]')", + "Plan": [ + "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path_string\"), cast(\"[\"a\", \"b\", \"c\"]\", json BINARY))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x61,0x61], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x62,0x62], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x63,0x63], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_string) */ * from t where json_overlaps((j0->'$.path_string'), '[\"a\", \"b\", \"c\"]') and a<10", + "Plan": [ + "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path_string\"), cast(\"[\"a\", \"b\", \"c\"]\", json BINARY))", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x61,0x61], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x62,0x62], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_string(cast(json_extract(`j0`, _utf8mb4'$.path_string') as char(10) array)) range:[0x63,0x63], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_date) */ * from t where (\"2023-01-01\" member of (j0->'$.path_date'))", + "Plan": [ + "Selection 8.00 root json_memberof(cast(\"2023-01-01\", json BINARY), json_extract(test.t.j0, \"$.path_date\"))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-01,2023-01-01], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_date) */ * from t where (\"2023-01-01\" member of (j0->'$.path_date')) and a<10", + "Plan": [ + "Selection 8.00 root json_memberof(cast(\"2023-01-01\", json BINARY), json_extract(test.t.j0, \"$.path_date\"))", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-01,2023-01-01], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_date) */ * from t where json_contains((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date)))", + "Plan": [ + "IndexMerge 10.00 root type: intersection", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-01,2023-01-01], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-02,2023-01-02], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-03,2023-01-03], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_date) */ * from t where json_contains((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date))) and a<10", + "Plan": [ + "IndexMerge 3.32 root type: intersection", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-01,2023-01-01], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-02,2023-01-02], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-03,2023-01-03], keep order:false, stats:pseudo", + "└─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_date) */ * from t where json_overlaps((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date)))", + "Plan": [ + "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path_date\"), json_array(cast(2023-01-01, json BINARY), cast(2023-01-02, json BINARY), cast(2023-01-03, json BINARY)))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-01,2023-01-01], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-02,2023-01-02], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-03,2023-01-03], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_date) */ * from t where json_overlaps((j0->'$.path_date'), json_array(cast('2023-01-01' as date), cast('2023-01-02' as date), cast('2023-01-03' as date))) and a<10", + "Plan": [ + "Selection 8.00 root json_overlaps(json_extract(test.t.j0, \"$.path_date\"), json_array(cast(2023-01-01, json BINARY), cast(2023-01-02, json BINARY), cast(2023-01-03, json BINARY)))", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-01,2023-01-01], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-02,2023-01-02], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_date(cast(json_extract(`j0`, _utf8mb4'$.path_date') as date array)) range:[2023-01-03,2023-01-03], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestCompositeMVIndex", + "Cases": [ + { + "SQL": "select /*+ use_index_merge(t, idx) */ * from t where a=1 and b=2 and (3 member of (j)) and c=4", + "Plan": [ + "Selection 0.00 root json_memberof(cast(3, json BINARY), test.t.j)", + "└─IndexMerge 0.00 root type: union", + " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2 3 4,1 2 3 4], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, idx) */ * from t where a=1 and b=2 and (3 member of (j))", + "Plan": [ + "Selection 0.08 root json_memberof(cast(3, json BINARY), test.t.j)", + "└─IndexMerge 0.00 root type: union", + " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2 3,1 2 3], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, idx) */ * from t where a=1 and b=2", + "Plan": [ + "IndexMerge 0.10 root type: union", + "├─IndexRangeScan(Build) 0.10 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2,1 2], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 0.10 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, idx) */ * from t where a=1", + "Plan": [ + "IndexMerge 10.00 root type: union", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1,1], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, idx2) */ * from t where a=1 and b=2 and ('3' member of (j->'$.str')) and c=4", + "Plan": [ + "Selection 0.00 root json_memberof(cast(\"3\", json BINARY), json_extract(test.t.j, \"$.str\"))", + "└─IndexMerge 0.00 root type: union", + " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(json_extract(`j`, _utf8mb4'$.str') as char(10) array), c) range:[1 2 0x33 4,1 2 0x33 4], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, idx2) */ * from t where a=1 and b=2 and ('3' member of (j->'$.str'))", + "Plan": [ + "Selection 0.08 root json_memberof(cast(\"3\", json BINARY), json_extract(test.t.j, \"$.str\"))", + "└─IndexMerge 0.00 root type: union", + " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(json_extract(`j`, _utf8mb4'$.str') as char(10) array), c) range:[1 2 0x33,1 2 0x33], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, idx2) */ * from t where a=1 and b=2", + "Plan": [ + "IndexMerge 0.10 root type: union", + "├─IndexRangeScan(Build) 0.10 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2,1 2], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 0.10 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, idx2) */ * from t where a=1", + "Plan": [ + "IndexMerge 10.00 root type: union", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1,1], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index(t, idx) */ * from t where a=1 and b=2 and (3 member of (j)) and c=4", + "Plan": [ + "Selection 0.00 root json_memberof(cast(3, json BINARY), test.t.j)", + "└─IndexMerge 0.00 root type: union", + " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2 3 4,1 2 3 4], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from t use index(idx) where a=1 and b=2 and (3 member of (j))", + "Plan": [ + "Selection 0.08 root json_memberof(cast(3, json BINARY), test.t.j)", + "└─IndexMerge 0.00 root type: union", + " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2 3,1 2 3], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index(t, idx) */ * from t where a=1 and b=2", + "Plan": [ + "IndexMerge 0.10 root type: union", + "├─IndexRangeScan(Build) 0.10 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2,1 2], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 0.10 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from t use index(idx) where a=1", + "Plan": [ + "IndexMerge 10.00 root type: union", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1,1], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from t force index(idx) where a=1 and b=2 and (3 member of (j))", + "Plan": [ + "Selection 0.08 root json_memberof(cast(3, json BINARY), test.t.j)", + "└─IndexMerge 0.00 root type: union", + " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1 2 3,1 2 3], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from t force index(idx) where a=1", + "Plan": [ + "IndexMerge 10.00 root type: union", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx(a, b, cast(`j` as signed array), c) range:[1,1], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestDNFOnMVIndex", + "Cases": [ + { + "SQL": "select /*+ use_index_merge(t, idx1) */ * from t where (1 member of (j)) or (2 member of (j))", + "Plan": [ + "Selection 8.00 root or(json_memberof(cast(1, json BINARY), test.t.j), json_memberof(cast(2, json BINARY), test.t.j))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[1,1], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[2,2], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, idx1) */ * from t where ((1 member of (j)) or (2 member of (j))) and (a > 10)", + "Plan": [ + "Selection 8.00 root or(json_memberof(cast(1, json BINARY), test.t.j), json_memberof(cast(2, json BINARY), test.t.j))", + "└─IndexMerge 3.33 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[1,1], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[2,2], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.33 cop[tikv] gt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, idx1) */ * from t where (json_overlaps(j, '[1, 2]')) or (json_overlaps(j, '[3, 4]'))", + "Plan": [ + "Selection 8.00 root or(json_overlaps(test.t.j, cast(\"[1, 2]\", json BINARY)), json_overlaps(test.t.j, cast(\"[3, 4]\", json BINARY)))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[1,1], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[2,2], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[3,3], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[4,4], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, idx1) */ * from t where ((json_overlaps(j, '[1, 2]')) or (json_overlaps(j, '[3, 4]'))) and (a > 10)", + "Plan": [ + "Selection 8.00 root or(json_overlaps(test.t.j, cast(\"[1, 2]\", json BINARY)), json_overlaps(test.t.j, cast(\"[3, 4]\", json BINARY)))", + "└─IndexMerge 3.33 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[1,1], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[2,2], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[3,3], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx1(cast(`j` as signed array)) range:[4,4], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.33 cop[tikv] gt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, idx1) */ * from t where (json_contains(j, '[1, 2]')) or (json_contains(j, '[3, 4]'))", + "Plan": [ + "TableReader 9600.00 root data:Selection", + "└─Selection 9600.00 cop[tikv] or(json_contains(test.t.j, cast(\"[1, 2]\", json BINARY)), json_contains(test.t.j, cast(\"[3, 4]\", json BINARY)))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, idx2) */ * from t where (a=1 and b=2 and (3 member of (j))) or (a=11 and b=12 and (13 member of (j)))", + "Plan": [ + "Selection 0.00 root or(and(eq(test.t.a, 1), and(eq(test.t.b, 2), json_memberof(cast(3, json BINARY), test.t.j))), and(eq(test.t.a, 11), and(eq(test.t.b, 12), json_memberof(cast(13, json BINARY), test.t.j))))", + "└─IndexMerge 0.00 root type: union", + " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(`j` as signed array), c) range:[1 2 3,1 2 3], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(`j` as signed array), c) range:[11 12 13,11 12 13], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, idx2) */ * from t where (a=1 and b=2 and (3 member of (j))) or (a=11 and b=12 and (13 member of (j)) and c=14)", + "Plan": [ + "Selection 0.00 root or(and(eq(test.t.a, 1), and(eq(test.t.b, 2), json_memberof(cast(3, json BINARY), test.t.j))), and(and(eq(test.t.a, 11), eq(test.t.b, 12)), and(json_memberof(cast(13, json BINARY), test.t.j), eq(test.t.c, 14))))", + "└─IndexMerge 0.00 root type: union", + " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(`j` as signed array), c) range:[1 2 3,1 2 3], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(`j` as signed array), c) range:[11 12 13 14,11 12 13 14], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, idx2) */ * from t where ((a=1 and b=2 and (3 member of (j))) or (a=11 and b=12 and (13 member of (j)))) and (c > 10)", + "Plan": [ + "Selection 0.00 root or(and(eq(test.t.a, 1), and(eq(test.t.b, 2), json_memberof(cast(3, json BINARY), test.t.j))), and(eq(test.t.a, 11), and(eq(test.t.b, 12), json_memberof(cast(13, json BINARY), test.t.j))))", + "└─IndexMerge 0.00 root type: union", + " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(`j` as signed array), c) range:[1 2 3,1 2 3], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:idx2(a, b, cast(`j` as signed array), c) range:[11 12 13,11 12 13], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.00 cop[tikv] gt(test.t.c, 10)", + " └─TableRowIDScan 0.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestMVIndexSelection", + "Cases": [ + { + "SQL": "select (j->'$.int') from t where (1 member of (j->'$.int'))", + "Plan": [ + "Projection 8000.00 root json_extract(test.t.j, $.int)->Column#5", + "└─Selection 8000.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j, \"$.int\"))", + " └─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from t where (1 member of (j->'$.int'))", + "Plan": [ + "Selection 8000.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j, \"$.int\"))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from t where (1 member of (j->'$.int')) and a<10", + "Plan": [ + "Selection 2658.67 root json_memberof(cast(1, json BINARY), json_extract(test.t.j, \"$.int\"))", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select (j->'$.int') from t where json_contains((j->'$.int'), '[1, 2, 3]')", + "Plan": [ + "Projection 8000.00 root json_extract(test.t.j, $.int)->Column#5", + "└─IndexMerge 10.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[2,2], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[3,3], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from t where json_contains((j->'$.int'), '[1, 2, 3]')", + "Plan": [ + "IndexMerge 10.00 root type: intersection", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[2,2], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[3,3], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from t where json_contains((j->'$.int'), '[1, 2, 3]') and a<10", + "Plan": [ + "IndexMerge 3.32 root type: intersection", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[2,2], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[3,3], keep order:false, stats:pseudo", + "└─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select (j->'$.int') from t where json_overlaps((j->'$.int'), '[1, 2, 3]')", + "Plan": [ + "Projection 8000.00 root json_extract(test.t.j, $.int)->Column#5", + "└─Selection 8000.00 root json_overlaps(json_extract(test.t.j, \"$.int\"), cast(\"[1, 2, 3]\", json BINARY))", + " └─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[2,2], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[3,3], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from t where json_overlaps((j->'$.int'), '[1, 2, 3]')", + "Plan": [ + "Selection 8000.00 root json_overlaps(json_extract(test.t.j, \"$.int\"), cast(\"[1, 2, 3]\", json BINARY))", + "└─IndexMerge 10.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[2,2], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[3,3], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from t where json_overlaps((j->'$.int'), '[1, 2, 3]') and a<10", + "Plan": [ + "Selection 2658.67 root json_overlaps(json_extract(test.t.j, \"$.int\"), cast(\"[1, 2, 3]\", json BINARY))", + "└─IndexMerge 3.32 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[2,2], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:i_int(cast(json_extract(`j`, _utf8mb4'$.int') as signed array)) range:[3,3], keep order:false, stats:pseudo", + " └─Selection(Probe) 3.32 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIndexMergePathGeneration", + "Cases": [ + "[]", + "[]", + "[{Idxs:[c_d_e,f],TbFilters:[]}]", + "[{Idxs:[c_d_e,f],TbFilters:[or(gt(test.t.c, 5), lt(test.t.f, 7))]},{Idxs:[c_d_e,f],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2))]}]", + "[{Idxs:[c_d_e,f],TbFilters:[or(gt(test.t.c, 5), lt(test.t.f, 7)),or(lt(test.t.c, 1), gt(test.t.g, 2))]},{Idxs:[c_d_e,f],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2)),or(lt(test.t.c, 1), gt(test.t.g, 2))]},{Idxs:[c_d_e,g],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2)),or(gt(test.t.c, 5), lt(test.t.f, 7))]}]", + "[{Idxs:[c_d_e,f],TbFilters:[or(gt(test.t.c, 5), lt(test.t.f, 7)),or(lt(test.t.e, 1), gt(test.t.f, 2))]},{Idxs:[c_d_e,f],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2)),or(lt(test.t.e, 1), gt(test.t.f, 2))]}]" + ] + }, + { + "Name": "TestHintForIntersectionIndexMerge", + "Cases": [ + { + "SQL": "set @@tidb_partition_prune_mode = 'dynamic'", + "Plan": null, + "Result": null + }, + { + "SQL": "select * from vh", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v1@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v1 where c < 30 and d in (2,5)", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ use_index_merge(t2, ia, ibc, id, ie) */ * from t2 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "Plan": [ + "IndexMerge 0.89 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:ia(a) range:(10,+inf], keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t2, index:id(d) range:[-inf,45), keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:ie(e) range:[100,100], keep order:false", + "└─TableRowIDScan(Probe) 0.89 cop[tikv] table:t2 keep order:false" + ], + "Result": [ + "20 20 20 5 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t3, ia, ibc, id, ie) */ * from t3 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "Plan": [ + "IndexMerge 0.44 root partition:p0 type: intersection", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t3, index:ia(a) range:(10,+inf], keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t3, index:id(d) range:[-inf,45), keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, index:ie(e) range:[100,100], keep order:false", + "└─TableRowIDScan(Probe) 0.44 cop[tikv] table:t3 keep order:false" + ], + "Result": [ + "20 20 20 5 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t4, ia, ibc, id, ie) */ * from t4 where a > 10 and b = 20 and c < 35 and d in (1,3,8,9) and e = 100", + "Plan": [ + "IndexMerge 0.89 root partition:p0,p1 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, index:ia(a) range:(10,+inf], keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t4, index:id(d) range:[1,1], [3,3], [8,8], [9,9], keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, index:ie(e) range:[100,100], keep order:false", + "└─TableRowIDScan(Probe) 0.89 cop[tikv] table:t4 keep order:false" + ], + "Result": [ + "30 20 5 8 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t5, is1, is2, is3, is4) */ * from t5 where s1 = 'Abc' and s2 > 'zzz' and s3 < 'B啊a' and s4 = 'CcC'", + "Plan": [ + "IndexMerge 0.00 root type: intersection", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t5, index:is1(s1) range:[\"Abc\",\"Abc\"], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t5, index:is2(s2) range:(\"zzz\",+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t5, index:is3(s3) range:[-inf,\"B啊a\"), keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t5, index:is4(s4) range:[\"CcC\",\"CcC\"], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 0.00 cop[tikv] table:t5 keep order:false, stats:pseudo" + ], + "Result": [ + "Abc zzzz aa ccc" + ] + }, + { + "SQL": "select /*+ use_index_merge(t6, primary, is3, is4) */ * from t6 where s1 = 'Abc' and s2 > 'zzz' and s3 = 'A啊a' and s4 not like 'Cd_'", + "Plan": [ + "IndexMerge 0.03 root type: intersection", + "├─IndexRangeScan(Build) 33.33 cop[tikv] table:t6, index:PRIMARY(s1, s2) range:(\"Abc\" \"zzz\",\"Abc\" +inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t6, index:is3(s3) range:[\"A啊a\",\"A啊a\"], keep order:false, stats:pseudo", + "└─Selection(Probe) 0.03 cop[tikv] gt(test.t6.s2, \"zzz\"), not(like(test.t6.s4, \"Cd_\", 92))", + " └─TableRowIDScan 0.03 cop[tikv] table:t6 keep order:false, stats:pseudo" + ], + "Result": [ + "Abc zzzz A啊A Cdaa" + ] + }, + { + "SQL": "select /*+ use_index_merge(t7, primary,ia,ib,ic,ie,iff,ig) */ * from t7 where a = 100 and b > 5 and c < 12.3 and d > 54.321 and e = '2022-11-22 17:00' and f > '2020-6-23 10:00' and g < 2025", + "Plan": [ + "IndexMerge 0.00 root type: intersection", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t7, index:PRIMARY(d) range:(54.321,+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t7, index:ia(a) range:[100,100], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t7, index:ib(b) range:(\"0x05\",+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t7, index:ic(c) range:[-inf,12.3), keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t7, index:ie(e) range:[2022-11-22 17:00:00,2022-11-22 17:00:00], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t7, index:iff(f) range:(2020-06-23 10:00:00.00000,+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t7, index:ig(g) range:[-inf,2025), keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 0.00 cop[tikv] table:t7 keep order:false, stats:pseudo" + ], + "Result": [ + "100 \u0006 12.2 56.000 2022-11-22 17:00:00 2022-12-21 00:00:00.00000 2021" + ] + }, + { + "SQL": "select /*+ use_index_merge(t8, primary,is2,is3,is4,is5) */ * from t8 where s1 like '啊A%' and s2 > 'abc' and s3 > 'cba' and s4 in ('aA', '??') and s5 = 'test,2'", + "Plan": [ + "Selection 1.42 root eq(test.t8.s5, \"test,2\")", + "└─IndexMerge 0.59 root type: intersection", + " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t8, index:is2(s2) range:(0x616263,+inf], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t8, index:is3(s3) range:(0x636261,+inf], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t8, index:is4(s4) range:[\"aA\",\"aA\"], [\"??\",\"??\"], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.59 cop[tikv] gt(test.t8.s3, \"cba\"), like(test.t8.s1, \"啊A%\", 92)", + " └─TableRowIDScan 2.22 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Result": [ + "啊aabbccdd abcc cccc aA tEsT,2" + ] + }, + { + "SQL": "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.c = t2.a) from t2", + "Plan": [ + "Projection 3.00 root test.t1.a", + "└─Apply 3.00 root CARTESIAN left outer join", + " ├─IndexReader(Build) 3.00 root partition:all index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:ia(a) keep order:false", + " └─MaxOneRow(Probe) 3.00 root ", + " └─IndexMerge 1.50 root partition:all type: intersection", + " ├─IndexRangeScan(Build) 6.00 cop[tikv] table:t1, index:ia(a) range:(10,+inf], keep order:false", + " ├─IndexRangeScan(Build) 4.00 cop[tikv] table:t1, index:ibc(b, c) range: decided by [eq(test.t1.b, 20) eq(test.t1.c, test.t2.a)], keep order:false", + " └─TableRowIDScan(Probe) 1.50 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "20", + "", + "" + ] + }, + { + "SQL": "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.c > t2.a) from t2", + "Plan": [ + "Projection 3.00 root test.t1.a", + "└─Apply 3.00 root CARTESIAN left outer join", + " ├─IndexReader(Build) 3.00 root partition:all index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:ia(a) keep order:false", + " └─MaxOneRow(Probe) 3.00 root ", + " └─IndexMerge 3.60 root partition:all type: intersection", + " ├─IndexRangeScan(Build) 6.00 cop[tikv] table:t1, index:ia(a) range:(10,+inf], keep order:false", + " ├─Selection(Build) 7.20 cop[tikv] gt(test.t1.c, test.t2.a)", + " │ └─IndexRangeScan 9.00 cop[tikv] table:t1, index:ibc(b, c) range:[20,20], keep order:false", + " └─TableRowIDScan(Probe) 3.60 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "20", + "20", + "" + ] + }, + { + "SQL": "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.e > t2.a) from t2", + "Plan": [ + "Projection 3.00 root test.t1.a", + "└─Apply 3.00 root CARTESIAN left outer join", + " ├─IndexReader(Build) 3.00 root partition:all index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:ia(a) keep order:false", + " └─MaxOneRow(Probe) 3.00 root ", + " └─IndexMerge 3.60 root partition:all type: intersection", + " ├─IndexRangeScan(Build) 6.00 cop[tikv] table:t1, index:ia(a) range:(10,+inf], keep order:false", + " ├─IndexRangeScan(Build) 9.00 cop[tikv] table:t1, index:ibc(b, c) range:[20,20], keep order:false", + " └─Selection(Probe) 3.60 cop[tikv] gt(test.t1.e, test.t2.a)", + " └─TableRowIDScan 4.50 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "20", + "20", + "20" + ] + }, + { + "SQL": "set @@tidb_partition_prune_mode = 'static'", + "Plan": null, + "Result": null + }, + { + "SQL": "select * from vh", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ use_index_merge(t2, ia, ibc, id, ie) */ * from t2 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "Plan": [ + "PartitionUnion 1.25 root ", + "├─IndexMerge 0.25 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p0, index:ia(a) range:(10,+inf], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p0, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, partition:p0, index:id(d) range:[-inf,45), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p0, index:ie(e) range:[100,100], keep order:false", + "│ └─TableRowIDScan(Probe) 0.25 cop[tikv] table:t2, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:ia(a) range:(10,+inf], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:id(d) range:[-inf,45), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:ie(e) range:[100,100], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t2, partition:p2, index:ia(a) range:(10,+inf], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t2, partition:p2, index:ibc(b, c) range:[20 -inf,20 35), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t2, partition:p2, index:id(d) range:[-inf,45), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, partition:p2, index:ie(e) range:[100,100], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "20 20 20 5 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t3, ia, ibc, id, ie) */ * from t3 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "Plan": [ + "IndexMerge 0.50 root type: intersection", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t3, partition:p0, index:ia(a) range:(10,+inf], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t3, partition:p0, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, partition:p0, index:id(d) range:[-inf,45), keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, partition:p0, index:ie(e) range:[100,100], keep order:false", + "└─TableRowIDScan(Probe) 0.50 cop[tikv] table:t3, partition:p0 keep order:false" + ], + "Result": [ + "20 20 20 5 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t4, ia, ibc, id, ie) */ * from t4 where a > 10 and b = 20 and c < 35 and d in (1,3,8,9) and e = 100", + "Plan": [ + "PartitionUnion 1.25 root ", + "├─IndexMerge 0.25 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p0, index:ia(a) range:(10,+inf], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p0, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, partition:p0, index:id(d) range:[1,1], [3,3], [8,8], [9,9], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p0, index:ie(e) range:[100,100], keep order:false", + "│ └─TableRowIDScan(Probe) 0.25 cop[tikv] table:t4, partition:p0 keep order:false", + "└─IndexMerge 1.00 root type: intersection", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:ia(a) range:(10,+inf], keep order:false", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:id(d) range:[1,1], [3,3], [8,8], [9,9], keep order:false", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:ie(e) range:[100,100], keep order:false", + " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t4, partition:p1 keep order:false" + ], + "Result": [ + "30 20 5 8 100" + ] + } + ] + } +] From 3580399e4565b1864f96c21ae200dde36f9fc4cd Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 15 Feb 2023 15:27:46 +0800 Subject: [PATCH 12/12] *: improve store bazel config Signed-off-by: Weizhen Wang --- planner/core/internal/BUILD.bazel | 5 ++++- planner/core/internal/testkit.go | 13 ------------- planner/core/internal/util.go | 31 +++++++++++++++++++++++++++++++ 3 files changed, 35 insertions(+), 14 deletions(-) create mode 100644 planner/core/internal/util.go diff --git a/planner/core/internal/BUILD.bazel b/planner/core/internal/BUILD.bazel index 3c4c45696c658..b57b813b5f190 100644 --- a/planner/core/internal/BUILD.bazel +++ b/planner/core/internal/BUILD.bazel @@ -2,7 +2,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "internal", - srcs = ["testkit.go"], + srcs = [ + "testkit.go", + "util.go", + ], importpath = "github.com/pingcap/tidb/planner/core/internal", visibility = ["//planner/core:__subpackages__"], deps = [ diff --git a/planner/core/internal/testkit.go b/planner/core/internal/testkit.go index 8e8d70dc0f9d1..4faa81cc0a8cf 100644 --- a/planner/core/internal/testkit.go +++ b/planner/core/internal/testkit.go @@ -21,9 +21,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/stretchr/testify/require" @@ -45,17 +43,6 @@ func SetTiFlashReplica(t *testing.T, dom *domain.Domain, dbName, tableName strin } } -// WrapCastForAggFuncs wraps the args of an aggregate function with a cast function. -// If the mode is FinalMode or Partial2Mode, we do not need to wrap cast upon the args, -// since the types of the args are already the expected. -func WrapCastForAggFuncs(sctx sessionctx.Context, aggFuncs []*aggregation.AggFuncDesc) { - for i := range aggFuncs { - if aggFuncs[i].Mode != aggregation.FinalMode && aggFuncs[i].Mode != aggregation.Partial2Mode { - aggFuncs[i].WrapCastForAggArgs(sctx) - } - } -} - // WithMockTiFlash sets the mockStore to have N TiFlash stores (naming as tiflash0, tiflash1, ...). func WithMockTiFlash(nodes int) mockstore.MockTiKVStoreOption { return mockstore.WithMultipleOptions( diff --git a/planner/core/internal/util.go b/planner/core/internal/util.go new file mode 100644 index 0000000000000..8bc9c3b5b66e9 --- /dev/null +++ b/planner/core/internal/util.go @@ -0,0 +1,31 @@ +// 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 internal + +import ( + "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/sessionctx" +) + +// WrapCastForAggFuncs wraps the args of an aggregate function with a cast function. +// If the mode is FinalMode or Partial2Mode, we do not need to wrap cast upon the args, +// since the types of the args are already the expected. +func WrapCastForAggFuncs(sctx sessionctx.Context, aggFuncs []*aggregation.AggFuncDesc) { + for i := range aggFuncs { + if aggFuncs[i].Mode != aggregation.FinalMode && aggFuncs[i].Mode != aggregation.Partial2Mode { + aggFuncs[i].WrapCastForAggArgs(sctx) + } + } +}