diff --git a/pkg/planner/core/plan_cache_test.go b/pkg/planner/core/plan_cache_test.go new file mode 100644 index 0000000000000..76282a6fb3da8 --- /dev/null +++ b/pkg/planner/core/plan_cache_test.go @@ -0,0 +1,1810 @@ +// Copyright 2022 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_test + +import ( + "context" + "errors" + "fmt" + "math/rand" + "strings" + "sync" + "testing" + "time" + + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner" + plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/session" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/testkit/testdata" + "github.com/pingcap/tidb/pkg/types" + driver "github.com/pingcap/tidb/pkg/types/parser_driver" + "github.com/pingcap/tidb/pkg/util" + "github.com/stretchr/testify/require" +) + +func TestInitLRUWithSystemVar(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@session.tidb_prepared_plan_cache_size = 0") // MinValue: 1 + tk.MustQuery("select @@session.tidb_prepared_plan_cache_size").Check(testkit.Rows("1")) + sessionVar := tk.Session().GetSessionVars() + + lru := plannercore.NewLRUPlanCache(uint(sessionVar.PreparedPlanCacheSize), 0, 0, tk.Session(), false) + require.NotNil(t, lru) +} + +func TestNonPreparedPlanCachePlanString(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t (a int, b int, key(a))`) + tk.MustExec(`set @@tidb_enable_non_prepared_plan_cache=1`) + + ctx := tk.Session() + planString := func(sql string) string { + stmts, err := session.Parse(ctx, sql) + require.NoError(t, err) + stmt := stmts[0] + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(context.Background(), ctx, stmt, plannercore.WithPreprocessorReturn(ret)) + require.NoError(t, err) + p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) + require.NoError(t, err) + return plannercore.ToString(p) + } + + require.Equal(t, planString("select a from t where a < 1"), "IndexReader(Index(t.a)[[-inf,1)])") + require.Equal(t, planString("select a from t where a < 10"), "IndexReader(Index(t.a)[[-inf,10)])") // range 1 -> 10 + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + + require.Equal(t, planString("select * from t where b < 1"), "TableReader(Table(t)->Sel([lt(test.t.b, 1)]))") + require.Equal(t, planString("select * from t where b < 10"), "TableReader(Table(t)->Sel([lt(test.t.b, 10)]))") // filter 1 -> 10 + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) +} + +func TestNonPreparedPlanCacheInformationSchema(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_enable_non_prepared_plan_cache=1") + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable(), plannercore.MockUnsignedTable()}) + + stmt, err := p.ParseOneStmt("select avg(a),avg(b),avg(c) from t", "", "") + require.NoError(t, err) + err = plannercore.Preprocess(context.Background(), tk.Session(), stmt, plannercore.WithPreprocessorReturn(&plannercore.PreprocessorReturn{InfoSchema: is})) + require.NoError(t, err) // no error + _, _, err = planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) // no error + _, _, err = planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) // no error + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) +} + +func TestNonPreparedPlanTypeRandomly(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t1 (a int, b int, key(a))`) + tk.MustExec(`create table t2 (a varchar(8), b varchar(8), key(a))`) + tk.MustExec(`create table t3 (a double, b double, key(a))`) + tk.MustExec(`create table t4 (a decimal(4, 2), b decimal(4, 2), key(a))`) + tk.MustExec(`create table t5 (a year, b year, key(a))`) + tk.MustExec(`create table t6 (a date, b date, key(a))`) + tk.MustExec(`create table t7 (a datetime, b datetime, key(a))`) + + n := 30 + for i := 0; i < n; i++ { + tk.MustExec(fmt.Sprintf(`insert into t1 values (%v, %v)`, randNonPrepTypeVal(t, n, "int"), randNonPrepTypeVal(t, n, "int"))) + tk.MustExec(fmt.Sprintf(`insert into t2 values (%v, %v)`, randNonPrepTypeVal(t, n, "varchar"), randNonPrepTypeVal(t, n, "varchar"))) + tk.MustExec(fmt.Sprintf(`insert into t3 values (%v, %v)`, randNonPrepTypeVal(t, n, "double"), randNonPrepTypeVal(t, n, "double"))) + tk.MustExec(fmt.Sprintf(`insert into t4 values (%v, %v)`, randNonPrepTypeVal(t, n, "decimal"), randNonPrepTypeVal(t, n, "decimal"))) + // TODO: fix it later + //tk.MustExec(fmt.Sprintf(`insert into t5 values (%v, %v)`, randNonPrepTypeVal(t, n, "year"), randNonPrepTypeVal(t, n, "year"))) + tk.MustExec(fmt.Sprintf(`insert into t6 values (%v, %v)`, randNonPrepTypeVal(t, n, "date"), randNonPrepTypeVal(t, n, "date"))) + tk.MustExec(fmt.Sprintf(`insert into t7 values (%v, %v)`, randNonPrepTypeVal(t, n, "datetime"), randNonPrepTypeVal(t, n, "datetime"))) + } + + for i := 0; i < 200; i++ { + q := fmt.Sprintf(`select * from t%v where %v`, rand.Intn(7)+1, randNonPrepFilter(t, n)) + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + r0 := tk.MustQuery(q).Sort() // the first execution + tk.MustQuery(q).Sort().Check(r0.Rows()) // may hit the cache + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=0`) + tk.MustQuery(q).Sort().Check(r0.Rows()) // disable the non-prep cache + } +} + +func randNonPrepFilter(t *testing.T, scale int) string { + switch rand.Intn(4) { + case 0: // >= + return fmt.Sprintf(`a >= %v`, randNonPrepVal(t, scale)) + case 1: // < + return fmt.Sprintf(`a < %v`, randNonPrepVal(t, scale)) + case 2: // = + return fmt.Sprintf(`a = %v`, randNonPrepVal(t, scale)) + case 3: // in + return fmt.Sprintf(`a in (%v, %v)`, randNonPrepVal(t, scale), randNonPrepVal(t, scale)) + } + require.Error(t, errors.New("")) + return "" +} + +func randNonPrepVal(t *testing.T, scale int) string { + return randNonPrepTypeVal(t, scale, [7]string{"int", "varchar", "double", + "decimal", "year", "datetime", "date"}[rand.Intn(7)]) +} + +func randNonPrepTypeVal(t *testing.T, scale int, typ string) string { + switch typ { + case "int": + return fmt.Sprintf("%v", rand.Intn(scale)-(scale/2)) + case "varchar": + return fmt.Sprintf("'%v'", rand.Intn(scale)-(scale/2)) + case "double", "decimal": + return fmt.Sprintf("%v", float64(rand.Intn(scale)-(scale/2))/float64(10)) + case "year": + return fmt.Sprintf("%v", 2000+rand.Intn(scale)) + case "date": + return fmt.Sprintf("'2023-01-%02d'", rand.Intn(scale)+1) + case "timestamp", "datetime": + return fmt.Sprintf("'2023-01-01 00:00:%02d'", rand.Intn(scale)) + default: + require.Error(t, errors.New(typ)) + return "" + } +} + +func TestNonPreparedPlanCacheBasically(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t (a int, b int, c int, d int, key(b), key(c, d))`) + for i := 0; i < 20; i++ { + tk.MustExec(fmt.Sprintf("insert into t values (%v, %v, %v, %v)", i, rand.Intn(20), rand.Intn(20), rand.Intn(20))) + } + + queries := []string{ + "select * from t where a<10", + "select * from t where a<13 and b<15", + "select * from t where b=13", + "select * from t where c<8", + "select * from t where d>8", + "select * from t where c=8 and d>10", + "select * from t where a<12 and b<13 and c<12 and d>2", + "select * from t where a in (1, 2, 3)", + "select * from t where a<13 or b<15", + "select * from t where a<13 or b<15 and c=13", + "select * from t where a in (1, 2)", + "select * from t where a in (1, 2) and b in (1, 2, 3)", + "select * from t where a in (1, 2) and b < 15", + "select * from t where a between 1 and 10", + "select * from t where a between 1 and 10 and b < 15", + } + + for _, query := range queries { + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=0`) + resultNormal := tk.MustQuery(query).Sort() + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + tk.MustQuery(query) // first process + tk.MustQuery(query).Sort().Check(resultNormal.Rows()) // equal to the result without plan-cache + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) // this plan is from plan-cache + } +} + +func TestNonPreparedPlanCacheInternalSQL(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec("create table t(a int, index(a))") + tk.MustExec("set tidb_enable_non_prepared_plan_cache=1") + + tk.MustExec("select * from t where a=1") + tk.MustExec("select * from t where a=1") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnOthers) + tk.Session().GetSessionVars().InRestrictedSQL = true + tk.MustExecWithContext(ctx, "select * from t where a=1") + tk.MustQueryWithContext(ctx, "select @@last_plan_from_cache").Check(testkit.Rows("0")) + + tk.Session().GetSessionVars().InRestrictedSQL = false + tk.MustExec("select * from t where a=1") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) +} + +func TestIssue38269(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set @@tidb_enable_prepared_plan_cache=1`) + tk.MustExec("set @@tidb_enable_collect_execution_info=0") + tk.MustExec(`set @@tidb_opt_advanced_join_hint=0`) + tk.MustExec("use test") + tk.MustExec("create table t1(a int)") + tk.MustExec("create table t2(a int, b int, c int, index idx(a, b))") + tk.MustExec("prepare stmt1 from 'select /*+ inl_join(t2) */ * from t1 join t2 on t1.a = t2.a where t2.b in (?, ?, ?)'") + tk.MustExec("set @a = 10, @b = 20, @c = 30, @d = 40, @e = 50, @f = 60") + tk.MustExec("execute stmt1 using @a, @b, @c") + tk.MustExec("execute stmt1 using @d, @e, @f") + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.Contains(t, rows[6][4], "range: decided by [eq(test.t2.a, test.t1.a) in(test.t2.b, 40, 50, 60)]") +} + +func TestIssue38533(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int, key (a))") + tk.MustExec(`prepare st from "select /*+ use_index(t, a) */ a from t where a=? and a=?"`) + tk.MustExec(`set @a=1`) + tk.MustExec(`execute st using @a, @a`) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + plan := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.True(t, strings.Contains(plan[1][0].(string), "RangeScan")) // range-scan instead of full-scan + + tk.MustExec(`execute st using @a, @a`) + tk.MustExec(`execute st using @a, @a`) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) +} + +func TestInvalidRange(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int, key(a))") + tk.MustExec("prepare st from 'select * from t where a>? and a (select max(a) from t t2 where t2.b < t1.b and t2.b < ?)", []int{1}, "0", false}, // scala + {"select * from t t1 where t1.a > (select 1 from t t2 where t2.b", "<", ">=", "<=", "in", "is null"} + op := ops[rand.Intn(len(ops))] + if op == "in" { + return fmt.Sprintf("%v %v (%v, %v, %v)", col, op, v(), v(), v()) + } else if op == "is null" { + return fmt.Sprintf("%v %v", col, op) + } + return fmt.Sprintf("%v %v %v", col, op, v()) + } + var queries []string + for i := 0; i < 50; i++ { + queries = append(queries, fmt.Sprintf("select * from t1 where %v", f())) + queries = append(queries, fmt.Sprintf("select * from t1 where %v and %v", f(), f())) + queries = append(queries, fmt.Sprintf("select * from t1 where %v and %v and %v", f(), f(), f())) + queries = append(queries, fmt.Sprintf("select * from t1 where %v and %v and %v and %v", f(), f(), f(), f())) + queries = append(queries, fmt.Sprintf("select * from t1 where %v and %v or %v", f(), f(), f())) + queries = append(queries, fmt.Sprintf("select * from t1 where %v and %v or %v and %v", f(), f(), f(), f())) + queries = append(queries, fmt.Sprintf("select * from t2 where %v", f())) + queries = append(queries, fmt.Sprintf("select * from t2 where %v and %v", f(), f())) + queries = append(queries, fmt.Sprintf("select * from t2 where %v and %v and %v", f(), f(), f())) + queries = append(queries, fmt.Sprintf("select * from t2 where %v and %v and %v and %v", f(), f(), f(), f())) + queries = append(queries, fmt.Sprintf("select * from t2 where %v and %v or %v", f(), f(), f())) + queries = append(queries, fmt.Sprintf("select * from t2 where %v and %v or %v and %v", f(), f(), f(), f())) + } + return queries +} + +func planCacheIntConvertQueries(isNonPrep bool) []string { + cols := []string{"a", "b", "c", "d"} + ops := []string{"=", ">", "<", ">=", "<=", "in", "is null"} + v := func() string { + var val string + switch rand.Intn(3) { + case 0: + val = fmt.Sprintf("%v", 2000+rand.Intn(20)-10) + case 1: + val = fmt.Sprintf("%v.0", 2000+rand.Intn(20)-10) + default: + val = fmt.Sprintf("'%v'", 2000+rand.Intn(20)-10) + } + if !isNonPrep { + val = fmt.Sprintf("#?%v#", val) + } + return val + } + f := func() string { + col := cols[rand.Intn(len(cols))] + op := ops[rand.Intn(len(ops))] + if op == "is null" { + return fmt.Sprintf("%v is null", col) + } else if op == "in" { + if rand.Intn(2) == 0 { + return fmt.Sprintf("%v in (%v)", col, v()) + } + return fmt.Sprintf("%v in (%v, %v, %v)", col, v(), v(), v()) + } + return fmt.Sprintf("%v %v %v", col, op, v()) + } + fields := func() string { + var fs []string + for _, f := range []string{"a", "b", "c", "d"} { + if rand.Intn(4) == 0 { + continue + } + fs = append(fs, f) + } + if len(fs) == 0 { + return "*" + } + return strings.Join(fs, ", ") + } + var queries []string + for i := 0; i < 50; i++ { + queries = append(queries, fmt.Sprintf("select %v from t where %v", fields(), f())) + queries = append(queries, fmt.Sprintf("select %v from t where %v and %v", fields(), f(), f())) + queries = append(queries, fmt.Sprintf("select %v from t where %v and %v and %v", fields(), f(), f(), f())) + queries = append(queries, fmt.Sprintf("select %v from t where %v or %v", fields(), f(), f())) + queries = append(queries, fmt.Sprintf("select %v from t where %v or %v or %v", fields(), f(), f(), f())) + queries = append(queries, fmt.Sprintf("select %v from t where %v and %v or %v", fields(), f(), f(), f())) + } + return queries +} + +func planCacheIntConvertPrepareData(tk *testkit.TestKit) { + tk.MustExec(`use test`) + tk.MustExec(`drop table if exists t`) + tk.MustExec(`create table t(a int, b year, c double, d varchar(16), key(a), key(b), key(c))`) + vals := make([]string, 0, 50) + for i := 0; i < 50; i++ { + a := fmt.Sprintf("%v", 2000+rand.Intn(20)-10) + if rand.Intn(10) == 0 { + a = "null" + } + b := fmt.Sprintf("%v", 2000+rand.Intn(20)-10) + if rand.Intn(10) == 0 { + b = "null" + } + c := fmt.Sprintf("%v.0", 2000+rand.Intn(20)-10) + if rand.Intn(10) == 0 { + c = "null" + } + d := fmt.Sprintf("'%v'", 2000+rand.Intn(20)-10) + if rand.Intn(10) == 0 { + d = "null" + } + vals = append(vals, fmt.Sprintf("(%s, %s, %s, %s)", a, b, c, d)) + } + tk.MustExec("insert into t values " + strings.Join(vals, ",")) +} + +func planCacheIndexMergeQueries(isNonPrep bool) []string { + ops := []string{"=", ">", "<", ">=", "<=", "in", "mod", "is null"} + f := func(col string) string { + n := rand.Intn(20) - 10 + nStr := fmt.Sprintf("%v", n) + if !isNonPrep { + nStr = fmt.Sprintf("#?%v#", n) + } + + op := ops[rand.Intn(len(ops))] + if op == "in" { + switch rand.Intn(3) { + case 0: // 1 element + return fmt.Sprintf("%s %s (%s)", col, op, nStr) + case 1: // multiple same elements + return fmt.Sprintf("%s %s (%s, %s, %s)", col, op, nStr, nStr, nStr) + default: // multiple different elements + if isNonPrep { + return fmt.Sprintf("%s %s (%d, %d)", col, op, n, n+1) + } + return fmt.Sprintf("%s %s (#?%d#, #?%d#)", col, op, n, n+1) + } + } else if op == "mod" { // this filter cannot be used to build range + return fmt.Sprintf("mod(%s, %s)=0", col, nStr) + } else if op == "is null" { + return fmt.Sprintf("%s %s", col, op) + } else { + return fmt.Sprintf("%s %s %s", col, op, nStr) + } + } + fields := func() string { + switch rand.Intn(5) { + case 0: + return "a" + case 1: + return "a, b" + case 2: + return "a, c" + case 3: + return "d" + default: + return "*" + } + } + var queries []string + for i := 0; i < 50; i++ { + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b) */ %s from t where %s and %s", fields(), f("a"), f("b"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, c) */ %s from t where %s and %s", fields(), f("a"), f("c"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b, c) */ %s from t where %s and %s and %s", fields(), f("a"), f("b"), f("c"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b) */ %s from t where %s or %s", fields(), f("a"), f("b"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, c) */ %s from t where %s or %s", fields(), f("a"), f("c"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b, c) */ %s from t where %s or %s or %s", fields(), f("a"), f("b"), f("c"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b) */ %s from t where %s and %s and %s", fields(), f("a"), f("a"), f("b"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, c) */ %s from t where %s and %s and %s", fields(), f("a"), f("c"), f("c"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b, c) */ %s from t where %s and %s and %s and %s", fields(), f("a"), f("b"), f("b"), f("c"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b) */ %s from t where (%s and %s) or %s", fields(), f("a"), f("a"), f("b"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, c) */ %s from t where %s or (%s and %s)", fields(), f("a"), f("c"), f("c"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b, c) */ %s from t where %s or (%s and %s) or %s", fields(), f("a"), f("b"), f("b"), f("c"))) + } + return queries +} + +func planCacheIndexMergePrepareData(tk *testkit.TestKit) { + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, c int, d int, key(a), key(b), key(c))") + vals := make([]string, 0, 50) + v := func() string { + if rand.Intn(10) == 0 { + return "null" + } + return fmt.Sprintf("%d", rand.Intn(20)-10) + } + for i := 0; i < 50; i++ { + vals = append(vals, fmt.Sprintf("(%s, %s, %s, %s)", v(), v(), v(), v())) + } + tk.MustExec("insert into t values " + strings.Join(vals, ",")) +} + +func TestPlanCacheRandomCases(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + testRandomPlanCacheCases(t, planCacheIndexMergePrepareData, planCacheIndexMergeQueries) + testRandomPlanCacheCases(t, planCacheIntConvertPrepareData, planCacheIntConvertQueries) + testRandomPlanCacheCases(t, planCachePointGetPrepareData, planCachePointGetQueries) +} + +func testRandomPlanCacheCases(t *testing.T, + prepFunc func(tk *testkit.TestKit), + queryFunc func(isNonPrep bool) []string) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + prepFunc(tk) + + // prepared plan cache + for _, q := range queryFunc(true) { + tk.MustExec("set tidb_enable_non_prepared_plan_cache=0") + result1 := tk.MustQuery(q).Sort() + tk.MustExec("set tidb_enable_non_prepared_plan_cache=1") + result2 := tk.MustQuery(q).Sort() + require.True(t, result1.Equal(result2.Rows())) + } + + // non prepared plan cache + for _, q := range queryFunc(false) { + q, prepStmt, parameters := convertQueryToPrepExecStmt(q) + result1 := tk.MustQuery(q).Sort() + tk.MustExec(prepStmt) + var xs []string + for i, p := range parameters { + tk.MustExec(fmt.Sprintf("set @x%d = %s", i, p)) + xs = append(xs, fmt.Sprintf("@x%d", i)) + } + var execStmt string + if len(xs) == 0 { + execStmt = "execute st" + } else { + execStmt = fmt.Sprintf("execute st using %s", strings.Join(xs, ", ")) + } + result2 := tk.MustQuery(execStmt).Sort() + require.True(t, result1.Equal(result2.Rows())) + } +} + +func TestPlanCacheSubquerySPMEffective(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)") + + testCases := []struct { + sql string + params []int + }{ + {"select * from t t1 where exists (select /*/ 1 from t t2 where t2.b < t1.b and t2.b < ?)", []int{1}}, // exist + {"select * from t t1 where exists (select /*/ b from t t2 where t1.a = t2.a and t2.b < ? limit ?)", []int{1, 1}}, + {"select * from t t1 where t1.a in (select /*/ a from t t2 where t2.a > ? and t1.a = t2.a)", []int{1}}, + {"select * from t t1 where t1.a < (select /*/ sum(t2.a) from t t2 where t2.b = t1.b and t2.a > ?)", []int{1}}, + } + + // hint + for _, testCase := range testCases { + sql := strings.Replace(testCase.sql, "/*/", "/*+ NO_DECORRELATE() */", 1) + tk.MustExec(fmt.Sprintf("prepare stmt from '%s'", sql)) + var using []string + for i, p := range testCase.params { + tk.MustExec(fmt.Sprintf("set @a%d = %d", i, p)) + using = append(using, fmt.Sprintf("@a%d", i)) + } + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + } + tk.MustExec("deallocate prepare stmt") + + // binding before prepare + for _, testCase := range testCases { + sql := strings.Replace(testCase.sql, "/*/", "", 1) + bindSQL := strings.Replace(testCase.sql, "/*/", "/*+ NO_DECORRELATE() */", 1) + tk.MustExec("create binding for " + sql + " using " + bindSQL) + tk.MustExec(fmt.Sprintf("prepare stmt from '%s'", sql)) + var using []string + for i, p := range testCase.params { + tk.MustExec(fmt.Sprintf("set @a%d = %d", i, p)) + using = append(using, fmt.Sprintf("@a%d", i)) + } + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + } + + // binding after prepare + for _, testCase := range testCases { + sql := strings.Replace(testCase.sql, "/*/", "", 1) + bindSQL := strings.Replace(testCase.sql, "/*/", "/*+ NO_DECORRELATE() */", 1) + tk.MustExec(fmt.Sprintf("prepare stmt from '%s'", sql)) + var using []string + for i, p := range testCase.params { + tk.MustExec(fmt.Sprintf("set @a%d = %d", i, p)) + using = append(using, fmt.Sprintf("@a%d", i)) + } + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustExec("create binding for " + sql + " using " + bindSQL) + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + } +} + +func TestIssue42125(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int, b int, c int, unique key(a, b))") + + // should use BatchPointGet + tk.MustExec("prepare st from 'select * from t where a=1 and b in (?, ?)'") + tk.MustExec("set @a=1, @b=2") + tk.MustExec("execute st using @a, @b") + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.Equal(t, rows[0][0], "Batch_Point_Get_5") // use BatchPointGet + tk.MustExec("execute st using @a, @b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip prepared plan-cache: Batch/PointGet plans may be over-optimized")) + + // should use PointGet: unsafe PointGet + tk.MustExec("prepare st from 'select * from t where a=1 and b>=? and b<=?'") + tk.MustExec("set @a=1, @b=1") + tk.MustExec("execute st using @a, @b") + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.Equal(t, rows[0][0], "Point_Get_5") // use Point_Get_5 + tk.MustExec("execute st using @a, @b") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // cannot hit + + // safe PointGet + tk.MustExec("prepare st from 'select * from t where a=1 and b=? and c Selection + require.Contains(t, rows[1][0], "Point_Get") + tk.MustExec("execute st using @a, @b") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) // can hit +} + +func TestNonPreparedPlanExplainWarning(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec(`create table t (a int, b int, c int, d int, e enum('1', '2', '3'), s set('1', '2', '3'), j json, bt bit(8), key(b), key(c, d))`) + tk.MustExec("create table t1(a int, b int, index idx_b(b)) partition by range(a) ( partition p0 values less than (6), partition p1 values less than (11) )") + tk.MustExec("create table t2(a int, b int) partition by hash(a) partitions 11") + tk.MustExec("create table t3(a int, first_name varchar(50), last_name varchar(50), full_name varchar(101) generated always as (concat(first_name,' ',last_name)))") + tk.MustExec("create or replace SQL SECURITY INVOKER view v as select a from t") + tk.MustExec("analyze table t, t1, t2") // eliminate stats warnings + tk.MustExec("set @@session.tidb_enable_non_prepared_plan_cache = 1") + + supported := []string{ + "select * from t where a<10", + "select * from t where a<13 and b<15", + "select * from t where b=13", + "select * from t where c<8", + "select * from t where d>8", + "select * from t where c=8 and d>10", + "select * from t where a<12 and b<13 and c<12 and d>2", + "select * from t where a in (1, 2, 3)", + "select * from t where a<13 or b<15", + "select * from t where a<13 or b<15 and c=13", + "select * from t where a in (1, 2)", + "select * from t where a in (1, 2) and b in (1, 2, 3)", + "select * from t where a in (1, 2) and b < 15", + "select * from t where a between 1 and 10", + "select * from t where a between 1 and 10 and b < 15", + "select * from t where a+b=13", + "select * from t where mod(a, 3)=1", + "select * from t where d>now()", + "select distinct a from t1 where a > 1 and b < 2", // distinct + "select count(*) from t1 where a > 1 and b < 2 group by a", // group by + "select * from t1 order by a", // order by + "select * from t3 where full_name = 'a b'", // generated column + "select * from t3 where a > 1 and full_name = 'a b'", + "select * from t1 where a in (1, 2)", // Partitioned + "select * from t2 where a in (1, 2) and b in (1, 2, 3)", // Partitioned + "select * from t1 where a in (1, 2) and b < 15", // Partitioned + } + + unsupported := []string{ + "select /*+ use_index(t1, idx_b) */ * from t1 where a > 1 and b < 2", // hint + "select a, sum(b) as c from t1 where a > 1 and b < 2 group by a having sum(b) > 1", // having + "select * from (select * from t1) t", // sub-query + "select * from t1 where a in (select a from t)", // uncorrelated sub-query + "select * from t1 where a in (select a from t where a > t1.a)", // correlated sub-query + "select * from t where j < 1", // json + "select * from t where a > 1 and j < 1", + "select * from t where e < '1'", // enum + "select * from t where a > 1 and e < '1'", + "select * from t where s < '1'", // set + "select * from t where a > 1 and s < '1'", + "select * from t where bt > 0", // bit + "select * from t where a > 1 and bt > 0", + "select data_type from INFORMATION_SCHEMA.columns where table_name = 'v'", // memTable + "select * from v", // view + "select * from t where a = null", // null + "select * from t where false", // table dual + } + + reasons := []string{ + "skip non-prepared plan-cache: queries that have hints, having-clause, window-function are not supported", + "skip non-prepared plan-cache: queries that have hints, having-clause, window-function are not supported", + "skip non-prepared plan-cache: queries that have sub-queries are not supported", + "skip non-prepared plan-cache: query has some unsupported Node", + "skip non-prepared plan-cache: query has some unsupported Node", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: access tables in system schema", + "skip non-prepared plan-cache: queries that access views are not supported", + "skip non-prepared plan-cache: query has null constants", + "skip non-prepared plan-cache: some parameters may be overwritten when constant propagation", + } + + all := append(supported, unsupported...) + + explainFormats := []string{ + types.ExplainFormatBrief, + types.ExplainFormatDOT, + types.ExplainFormatHint, + types.ExplainFormatROW, + types.ExplainFormatVerbose, + types.ExplainFormatTraditional, + types.ExplainFormatBinary, + types.ExplainFormatTiDBJSON, + types.ExplainFormatCostTrace, + } + // all cases no warnings use other format + for _, q := range all { + tk.MustExec("explain " + q) + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustExec("explain " + q) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + } + for _, format := range explainFormats { + for _, q := range all { + tk.MustExec(fmt.Sprintf("explain format = '%v' %v", format, q)) + //tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustQuery("show warnings").CheckNotContain("plan cache") + tk.MustExec(fmt.Sprintf("explain format = '%v' %v", format, q)) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + } + } + + // unsupported case with warning use 'plan_cache' format + for idx, q := range unsupported { + tk.MustExec("explain format = 'plan_cache'" + q) + warn := tk.MustQuery("show warnings").Rows()[0] + require.Equal(t, reasons[idx], warn[2], "idx: %d", idx) + } +} + +func TestNonPreparedPlanCachePanic(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + + tk.MustExec("create table t (a varchar(255), b int, c char(10), primary key (c, a));") + ctx := tk.Session().(sessionctx.Context) + + s := parser.New() + for _, sql := range []string{ + "select 1 from t where a='x'", + "select * from t where c='x'", + "select * from t where a='x' and c='x'", + "select * from t where a='x' and c='x' and b=1", + } { + stmtNode, err := s.ParseOneStmt(sql, "", "") + require.NoError(t, err) + preprocessorReturn := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(context.Background(), ctx, stmtNode, plannercore.WithPreprocessorReturn(preprocessorReturn)) + require.NoError(t, err) + _, _, err = planner.Optimize(context.TODO(), ctx, stmtNode, preprocessorReturn.InfoSchema) + require.NoError(t, err) // not panic + } +} + +func TestNonPreparedPlanCacheAutoStmtRetry(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk1.MustExec("create table t(id int primary key, k int, UNIQUE KEY(k))") + tk1.MustExec("insert into t values(1, 1)") + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + tk2.MustExec("use test") + tk1.MustExec("begin") + tk1.MustExec("update t set k=3 where id=1") + + var wg sync.WaitGroup + var tk2Err error + wg.Add(1) + go func() { + // trigger statement auto-retry on tk2 + _, tk2Err = tk2.Exec("insert into t values(3, 3)") + wg.Done() + }() + time.Sleep(100 * time.Millisecond) + _, err := tk1.Exec("commit") + require.NoError(t, err) + wg.Wait() + require.ErrorContains(t, tk2Err, "Duplicate entry") +} + +func TestIssue43667Concurrency(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table cycle (pk int key, val int)") + var wg sync.WaitGroup + concurrency := 30 + for i := 0; i < concurrency; i++ { + tk.MustExec(fmt.Sprintf("insert into cycle values (%v,%v)", i, i)) + wg.Add(1) + go func(id int) { + defer wg.Done() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_enable_non_prepared_plan_cache=1") + query := fmt.Sprintf("select (val) from cycle where pk = %v", id) + for j := 0; j < 5000; j++ { + tk.MustQuery(query).Check(testkit.Rows(fmt.Sprintf("%v", id))) + } + }(i) + } + wg.Wait() +} + +func TestIssue43667(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + tk.MustExec(`create table cycle (pk int not null primary key, sk int not null, val int)`) + tk.MustExec(`insert into cycle values (4, 4, 4)`) + tk.MustExec(`insert into cycle values (7, 7, 7)`) + + tk.MustQuery(`select (val) from cycle where pk = 4`).Check(testkit.Rows("4")) + tk.MustQuery(`select (val) from cycle where pk = 7`).Check(testkit.Rows("7")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + + updateAST := func(stmt ast.StmtNode) { + v := stmt.(*ast.SelectStmt).Where.(*ast.BinaryOperationExpr).R.(*driver.ValueExpr) + v.Datum.SetInt64(7) + } + + tctx := context.WithValue(context.Background(), plannercore.PlanCacheKeyTestIssue43667{}, updateAST) + tk.MustQueryWithContext(tctx, `select (val) from cycle where pk = 4`).Check(testkit.Rows("4")) +} + +func TestIssue47133(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + tk.MustExec(`CREATE TABLE t (id int NOT NULL, personId int NOT NULL, + name varchar(255) NOT NULL, PRIMARY KEY (id, personId))`) + tk.MustExec(`insert into t values (1, 1, '')`) + + cnt := 0 + checkFieldNames := func(names []*types.FieldName) { + require.Equal(t, len(names), 2) + require.Equal(t, names[0].String(), "test.t.user_id") + require.Equal(t, names[1].String(), "test.t.user_personid") + cnt += 1 + } + tctx := context.WithValue(context.Background(), plannercore.PlanCacheKeyTestIssue47133{}, checkFieldNames) + tk.MustQueryWithContext(tctx, `SELECT id AS User_id, personId AS User_personId FROM t WHERE (id = 1 AND personId = 1)`).Check( + testkit.Rows("1 1")) + tk.MustQueryWithContext(tctx, `SELECT id AS User_id, personId AS User_personId FROM t WHERE (id = 1 AND personId = 1)`).Check( + testkit.Rows("1 1")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + require.Equal(t, cnt, 2) +} + +func TestPlanCacheBindingIgnore(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create database test1`) + tk.MustExec(`use test1`) + tk.MustExec(`create table t (a int)`) + tk.MustExec(`create database test2`) + tk.MustExec(`use test2`) + tk.MustExec(`create table t (a int)`) + + tk.MustExec(`prepare st1 from 'select * from test1.t'`) + tk.MustExec(`execute st1`) + tk.MustExec(`execute st1`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustExec(`prepare st2 from 'select * from test2.t'`) + tk.MustExec(`execute st2`) + tk.MustExec(`execute st2`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + + tk.MustExec(`create global binding using select /*+ ignore_plan_cache() */ * from test1.t`) + tk.MustExec(`execute st1`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + tk.MustExec(`execute st1`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + tk.MustExec(`create global binding using select /*+ ignore_plan_cache() */ * from test2.t`) + tk.MustExec(`execute st2`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + tk.MustExec(`execute st2`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) +} + +func TestBuiltinFuncFlen(t *testing.T) { + // same as TestIssue45378 and TestIssue45253 + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`CREATE TABLE t1(c1 INT)`) + tk.MustExec(`INSERT INTO t1 VALUES (1)`) + + funcs := []string{ast.Abs, ast.Acos, ast.Asin, ast.Atan, ast.Ceil, ast.Ceiling, ast.Cos, + ast.CRC32, ast.Degrees, ast.Floor, ast.Ln, ast.Log, ast.Log2, ast.Log10, ast.Unhex, + ast.Radians, ast.Rand, ast.Round, ast.Sign, ast.Sin, ast.Sqrt, ast.Tan, ast.SM3, + ast.Quote, ast.RTrim, ast.ToBase64, ast.Trim, ast.Upper, ast.Ucase, ast.Hex, + ast.BitLength, ast.CharLength, ast.Compress, ast.MD5, ast.SHA1, ast.SHA} + args := []string{"2038330881", "'2038330881'", "'牵'", "-1", "''", "0"} + + for _, f := range funcs { + for _, a := range args { + q := fmt.Sprintf("SELECT c1 from t1 where %s(%s)", f, a) + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + r1 := tk.MustQuery(q) + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=0`) + r2 := tk.MustQuery(q) + r1.Sort().Check(r2.Sort().Rows()) + } + } +} + +func TestWarningWithDisablePlanCacheStmt(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int) partition by hash(a) partitions 4;") + tk.MustExec("analyze table t;") + tk.MustExec("prepare st from 'select * from t';") + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + tk.MustExec("execute st;") + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + tk.MustExec("execute st;") + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`show warnings`).Check(testkit.Rows()) +} + +func randValueForMVIndex(colType string) string { + randSize := 50 + colType = strings.ToLower(colType) + switch colType { + case "int": + return fmt.Sprintf("%v", randSize-rand.Intn(randSize)) + case "string": + return fmt.Sprintf("\"%v\"", rand.Intn(randSize)) + case "json-string": + var array []string + arraySize := 1 + rand.Intn(5) + for i := 0; i < arraySize; i++ { + array = append(array, randValueForMVIndex("string")) + } + return "'[" + strings.Join(array, ", ") + "]'" + case "json-signed": + var array []string + arraySize := 1 + rand.Intn(5) + for i := 0; i < arraySize; i++ { + array = append(array, randValueForMVIndex("int")) + } + return "'[" + strings.Join(array, ", ") + "]'" + default: + return "unknown type " + colType + } +} + +func insertValuesForMVIndex(nRows int, colTypes ...string) string { + var stmtVals []string + for i := 0; i < nRows; i++ { + var vals []string + for _, colType := range colTypes { + vals = append(vals, randValueForMVIndex(colType)) + } + stmtVals = append(stmtVals, "("+strings.Join(vals, ", ")+")") + } + return strings.Join(stmtVals, ", ") +} + +func verifyPlanCacheForMVIndex(t *testing.T, tk *testkit.TestKit, isIndexMerge, hitCache bool, queryTemplate string, colTypes ...string) { + for i := 0; i < 5; i++ { + var vals []string + for _, colType := range colTypes { + vals = append(vals, randValueForMVIndex(colType)) + } + + query := queryTemplate + var setStmt, usingStmt string + for i, p := range vals { + query = strings.Replace(query, "?", p, 1) + if i > 0 { + setStmt += ", " + usingStmt += ", " + } + setStmt += fmt.Sprintf("@a%v=%v", i, p) + usingStmt += fmt.Sprintf("@a%v", i) + } + result := tk.MustQuery(query).Sort() + if isIndexMerge { + tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning + } + tk.MustExec(fmt.Sprintf("set %v", setStmt)) + tk.MustExec(fmt.Sprintf("prepare stmt from '%v'", queryTemplate)) + if isIndexMerge { + tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning + } + result1 := tk.MustQuery(fmt.Sprintf("execute stmt using %v", usingStmt)).Sort() + result.Check(result1.Rows()) + if isIndexMerge && hitCache { + tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning + } + result2 := tk.MustQuery(fmt.Sprintf("execute stmt using %v", usingStmt)).Sort() + result.Check(result2.Rows()) + if isIndexMerge && hitCache { + tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning + } + result3 := tk.MustQuery(fmt.Sprintf("execute stmt using %v", usingStmt)).Sort() + result.Check(result3.Rows()) + if isIndexMerge && hitCache { + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) // hit the cache + } + + if isIndexMerge && hitCache { // check the plan + result4 := tk.MustQuery(fmt.Sprintf("execute stmt using %v", usingStmt)).Sort() + result.Check(result4.Rows()) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + haveIndexMerge := false + for _, r := range rows { + if strings.Contains(r[0].(string), "IndexMerge") { + haveIndexMerge = true + } + } + require.True(t, haveIndexMerge) // IndexMerge has to be used. + } + } +} + +func TestPlanCacheMVIndexRandomly(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`set @@tidb_opt_fix_control = "45798:on"`) + + // cases from TestIndexMergeFromComposedDNFCondition + tk.MustExec(`drop table if exists t2`) + tk.MustExec(`create table t2(a json, b json, c int, d int, e int, index idx(c, (cast(a as signed array))), index idx2((cast(b as signed array)), c), index idx3(c, d), index idx4(d))`) + tk.MustExec(fmt.Sprintf("insert into t2 values %v", insertValuesForMVIndex(100, "json-signed", "json-signed", "int", "int", "int"))) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t2, idx2, idx) */ * from t2 where (? member of (a) and c=?) or (? member of (b) and c=?)`, + `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t2, idx2, idx) */ * from t2 where (? member of (a) and c=? and d=?) or (? member of (b) and c=? and d=?)`, + `int`, `int`, `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, false, false, + `select /*+ use_index_merge(t2, idx2, idx) */ * from t2 where ( json_contains(a, ?) and c=? and d=?) or (? member of (b) and c=? and d=?)`, + `json-signed`, `int`, `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, false, + `select /*+ use_index_merge(t2, idx2, idx) */ * from t2 where ( json_overlaps(a, ?) and c=? and d=?) or (? member of (b) and c=? and d=?)`, + `json-signed`, `int`, `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t2, idx2, idx, idx4) */ * from t2 where ( json_contains(a, ?) and d=?) or (? member of (b) and c=? and d=?)`, + `json-signed`, `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t2, idx2, idx) */ * from t2 where (? member of (a) and ? member of (b) and c=?) or (? member of (b) and c=?)`, + `int`, `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, false, true, + `select * from t2 where (? member of (a) and ? member of (b) and c=?) or (? member of (b) and c=?) or e=?`, + `int`, `int`, `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t2, idx2, idx, idx4) */ * from t2 where (? member of (a) and ? member of (b) and c=?) or (? member of (b) and c=?) or d=?`, + `int`, `int`, `int`, `int`, `int`, `int`) + + // cases from TestIndexMergeFromComposedCNFCondition + tk.MustExec(`drop table if exists t1, t2`) + tk.MustExec(`create table t1(a json, b json, c int, d int, index idx((cast(a as signed array))), index idx2((cast(b as signed array))))`) + tk.MustExec(fmt.Sprintf("insert into t1 values %v", insertValuesForMVIndex(100, "json-signed", "json-signed", "int", "int"))) + tk.MustExec(`create table t2(a json, b json, c int, d int, index idx(c, (cast(a as signed array))), index idx2((cast(b as signed array)), c), index idx3(c, d), index idx4(d))`) + tk.MustExec(fmt.Sprintf("insert into t2 values %v", insertValuesForMVIndex(100, "json-signed", "json-signed", "int", "int"))) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t1, idx, idx2) */ * from t1 where ? member of (a) and ? member of (b)`, + `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t2, idx, idx2) */ * from t2 where ? member of (a) and ? member of (b) and c=?`, + `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t2, idx, idx2, idx4) */ * from t2 where ? member of (a) and ? member of (b) and c=? and d=?`, + `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, false, + `select /*+ use_index_merge(t2, idx2, idx, idx3) */ * from t2 where json_contains(a, ?) and c=? and ? member of (b) and d=?`, + `json-signed`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, false, + `select /*+ use_index_merge(t2, idx2, idx, idx3) */ * from t2 where json_overlaps(a, ?) and c=? and ? member of (b) and d=?`, + `json-signed`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, false, true, + `select /*+ use_index_merge(t2, idx2, idx) */ * from t2 where ? member of (a) and c=? and c=?`, + `int`, `int`, `int`) + + // case from TestIndexMergeIssue50265 + tk.MustExec(`drop table if exists t`) + tk.MustExec("create table t(pk varbinary(255) NOT NULL, domains json null, image_signatures json null, canonical_links json null, fpi json null, KEY `domains` ((cast(`domains` as char(253) array))), KEY `image_signatures` ((cast(`image_signatures` as char(32) array))),KEY `canonical_links` ((cast(`canonical_links` as char(1000) array))), KEY `fpi` ((cast(`fpi` as signed array))))") + tk.MustExec(fmt.Sprintf("insert into t values %v", insertValuesForMVIndex(100, "string", "json-string", "json-string", "json-string", "json-signed"))) + verifyPlanCacheForMVIndex(t, tk, false, false, + `SELECT /*+ use_index_merge(t, domains, image_signatures, canonical_links, fpi) */ pk FROM t WHERE ? member of (domains) OR ? member of (image_signatures) OR ? member of (canonical_links) OR json_contains(fpi, "[69236881]") LIMIT 100`, + `string`, `string`, `string`) + + // case from TestIndexMergeEliminateRedundantAndPaths + tk.MustExec(`DROP table if exists t`) + tk.MustExec("CREATE TABLE `t` (`pk` varbinary(255) NOT NULL,`nslc` json DEFAULT NULL,`fpi` json DEFAULT NULL,`point_of_sale_country` varchar(2) DEFAULT NULL,KEY `fpi` ((cast(`fpi` as signed array))),KEY `nslc` ((cast(`nslc` as char(1000) array)),`point_of_sale_country`),KEY `nslc_old` ((cast(`nslc` as char(1000) array))))") + tk.MustExec(fmt.Sprintf("insert into t values %v", insertValuesForMVIndex(100, "string", "json-string", "json-signed", "string"))) + verifyPlanCacheForMVIndex(t, tk, true, true, + "SELECT /*+ use_index_merge(t, fpi, nslc_old, nslc) */ * FROM t WHERE ? member of (fpi) AND ? member of (nslc) LIMIT 100", + "int", "string") + + // case from TestIndexMergeSingleCaseCouldFeelIndexMergeHint + tk.MustExec(`DROP table if exists t`) + tk.MustExec("CREATE TABLE t (nslc json DEFAULT NULL,fpi json DEFAULT NULL,point_of_sale_country int,KEY nslc ((cast(nslc as char(1000) array)),point_of_sale_country),KEY fpi ((cast(fpi as signed array))))") + tk.MustExec(fmt.Sprintf("insert into t values %v", insertValuesForMVIndex(100, "json-string", "json-signed", "int"))) + verifyPlanCacheForMVIndex(t, tk, true, true, + "SELECT /*+ use_index_merge(t, nslc) */ * FROM t WHERE ? member of (fpi) AND ? member of (nslc) LIMIT 1", + "int", "string") + verifyPlanCacheForMVIndex(t, tk, true, true, + "SELECT /*+ use_index_merge(t, fpi) */ * FROM t WHERE ? member of (fpi) AND ? member of (nslc) LIMIT 1", + "int", "string") +} + +func TestPlanCacheMVIndexManually(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`set @@tidb_opt_fix_control = "45798:on"`) + + var ( + input []string + output []struct { + SQL string + Result []string + } + ) + planSuiteData := plannercore.GetPlanCacheSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + for i := range input { + testdata.OnRecord(func() { + output[i].SQL = input[i] + }) + if strings.HasPrefix(strings.ToLower(input[i]), "select") || + strings.HasPrefix(strings.ToLower(input[i]), "execute") || + strings.HasPrefix(strings.ToLower(input[i]), "show") { + result := tk.MustQuery(input[i]) + testdata.OnRecord(func() { + output[i].Result = testdata.ConvertRowsToStrings(result.Rows()) + }) + result.Check(testkit.Rows(output[i].Result...)) + } else { + tk.MustExec(input[i]) + } + } +} + +func BenchmarkPlanCacheBindingMatch(b *testing.B) { + store := testkit.CreateMockStore(b) + tk := testkit.NewTestKit(b, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int, key(a))") + tk.MustExec(`create global binding using select * from t where a=1`) + + tk.MustExec(`prepare st from 'select * from t where a=?'`) + tk.MustExec(`set @a=1`) + b.ResetTimer() + for i := 0; i < b.N; i++ { + tk.MustExec("execute st using @a") + } +} + +func BenchmarkPlanCacheInsert(b *testing.B) { + store := testkit.CreateMockStore(b) + tk := testkit.NewTestKit(b, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int)") + + tk.MustExec("prepare st from 'insert into t values (1)'") + b.ResetTimer() + for i := 0; i < b.N; i++ { + tk.MustExec("execute st") + } +} + +func BenchmarkNonPreparedPlanCacheDML(b *testing.B) { + store := testkit.CreateMockStore(b) + tk := testkit.NewTestKit(b, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int)") + tk.MustExec("set tidb_enable_non_prepared_plan_cache=1") + + b.ResetTimer() + for i := 0; i < b.N; i++ { + tk.MustExec("insert into t values (1)") + tk.MustExec("update t set a = 2 where a = 1") + tk.MustExec("delete from t where a = 2") + } +} + +func TestPreparedPlanCachePartitions(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec(`create table t (a int primary key, b varchar(255)) partition by hash(a) partitions 3`) + tk.MustExec(`insert into t values (1,"a"),(2,"b"),(3,"c"),(4,"d"),(5,"e"),(6,"f")`) + tk.MustExec(`analyze table t`) + tk.MustExec(`prepare stmt from 'select a,b from t where a = ?;'`) + tk.MustExec(`set @a=1`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("1 a")) + // Same partition works, due to pruning is not affected + tk.MustExec(`set @a=4`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("4 d")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + // Different partition needs code changes + tk.MustExec(`set @a=2`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("2 b")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustExec(`prepare stmt2 from 'select b,a from t where a = ?;'`) + tk.MustExec(`set @a=1`) + tk.MustQuery(`execute stmt2 using @a`).Check(testkit.Rows("a 1")) + tk.MustExec(`set @a=3`) + tk.MustQuery(`execute stmt2 using @a`).Check(testkit.Rows("c 3")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustExec(`drop table t`) + + tk.MustExec(`create table t (a int primary key, b varchar(255), c varchar(255), key (b)) partition by range (a) (partition pNeg values less than (0), partition p0 values less than (1000000), partition p1M values less than (2000000))`) + tk.MustExec(`insert into t values (-10, -10, -10), (0, 0, 0), (-1, NULL, NULL), (1000, 1000, 1000), (1000000, 1000000, 1000000), (1500000, 1500000, 1500000), (1999999, 1999999, 1999999)`) + tk.MustExec(`analyze table t`) + tk.MustExec(`prepare stmt3 from 'select a,c,b from t where a = ?'`) + tk.MustExec(`set @a=2000000`) + // This should use TableDual + tk.MustQuery(`execute stmt3 using @a`).Check(testkit.Rows()) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + 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)).MultiCheckContain([]string{"Point_Get", "partition:dual", "handle:2000000"}) + tk.MustExec(`set @a=1999999`) + tk.MustQuery(`execute stmt3 using @a`).Check(testkit.Rows("1999999 1999999 1999999")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + 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)).MultiCheckContain([]string{"Point_Get", "partition:p1M", "handle:1999999"}) + tk.MustQuery(`execute stmt3 using @a`).Check(testkit.Rows("1999999 1999999 1999999")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + + tk.MustExec(`prepare stmt4 from 'select a,c,b from t where a IN (?,?,?)'`) + tk.MustExec(`set @a=1999999,@b=0,@c=-1`) + tk.MustQuery(`execute stmt4 using @a,@b,@c`).Sort().Check(testkit.Rows("-1 ", "0 0 0", "1999999 1999999 1999999")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`execute stmt4 using @a,@b,@c`).Sort().Check(testkit.Rows("-1 ", "0 0 0", "1999999 1999999 1999999")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) +} + +func TestPreparedPlanCachePartitionIndex(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`create table t (b varchar(255), a int primary key nonclustered, key (b)) partition by key(a) partitions 3`) + tk.MustExec(`insert into t values ('Ab', 1),('abc',2),('BC',3),('AC',4),('BA',5),('cda',6)`) + tk.MustExec(`analyze table t`) + tk.MustExec(`prepare stmt from 'select * from t where a IN (?,?,?)'`) + tk.MustExec(`set @a=1,@b=3,@c=4`) + tk.MustQuery(`execute stmt using @a,@b,@c`).Sort().Check(testkit.Rows("AC 4", "Ab 1", "BC 3")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`execute stmt using @a,@b,@c`).Sort().Check(testkit.Rows("AC 4", "Ab 1", "BC 3")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + 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)).CheckAt([]int{0}, [][]any{ + {"IndexLookUp_7"}, + {"├─IndexRangeScan_5(Build)"}, + {"└─TableRowIDScan_6(Probe)"}}) + tk.MustExec(`set @a=2,@b=5,@c=4`) + tk.MustQuery(`execute stmt using @a,@b,@c`).Sort().Check(testkit.Rows("AC 4", "BA 5", "abc 2")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) +} + +func TestNonPreparedPlanCachePartitionIndex(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set @@tidb_enable_non_prepared_plan_cache=1`) + tk.MustExec("use test") + tk.MustExec(`create table t (b varchar(255), a int primary key nonclustered, key (b)) partition by key(a) partitions 3`) + // [Batch]PointGet does not use the plan cache, + // since it is already using the fast path! + tk.MustExec(`insert into t values ('Ab', 1),('abc',2),('BC',3),('AC',4),('BA',5),('cda',6)`) + tk.MustExec(`analyze table t`) + tk.MustQuery(`explain format='plan_cache' select * from t where a IN (2,1,4,1,1,5,5)`).Check(testkit.Rows(""+ + "IndexLookUp_7 4.00 root partition:p1,p2 ", + "├─IndexRangeScan_5(Build) 4.00 cop[tikv] table:t, index:PRIMARY(a) range:[1,1], [2,2], [4,4], [5,5], keep order:false", + "└─TableRowIDScan_6(Probe) 4.00 cop[tikv] table:t keep order:false")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`select * from t where a IN (2,1,4,1,1,5,5)`).Sort().Check(testkit.Rows("AC 4", "Ab 1", "BA 5", "abc 2")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`select * from t where a IN (1,3,4)`).Sort().Check(testkit.Rows("AC 4", "Ab 1", "BC 3")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`select * from t where a IN (1,3,4)`).Sort().Check(testkit.Rows("AC 4", "Ab 1", "BC 3")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`select * from t where a IN (2,5,4,2,5,5,1)`).Sort().Check(testkit.Rows("AC 4", "Ab 1", "BA 5", "abc 2")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`select * from t where a IN (1,2,3,4,5,5,1)`).Sort().Check(testkit.Rows("AC 4", "Ab 1", "BA 5", "BC 3", "abc 2")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`select count(*) from t partition (p0)`).Check(testkit.Rows("0")) + tk.MustQuery(`select count(*) from t partition (p1)`).Check(testkit.Rows("5")) + tk.MustQuery(`select * from t partition (p2)`).Check(testkit.Rows("Ab 1")) + + tk.MustQuery(`explain format='plan_cache' select * from t where a = 2`).Check(testkit.Rows("Point_Get_1 1.00 root table:t, partition:p1, index:PRIMARY(a) ")) + tk.MustQuery(`explain format='plan_cache' select * from t where a = 2`).Check(testkit.Rows("Point_Get_1 1.00 root table:t, partition:p1, index:PRIMARY(a) ")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`select * from t where a = 2`).Check(testkit.Rows("abc 2")) + tk.MustExec(`create table tk (a int primary key nonclustered, b varchar(255), key (b)) partition by key (a) partitions 3`) + tk.MustExec(`insert into tk select a, b from t`) + tk.MustExec(`analyze table tk`) + tk.MustQuery(`explain format='plan_cache' select * from tk where a = 2`).Check(testkit.Rows("Point_Get_1 1.00 root table:tk, partition:p1, index:PRIMARY(a) ")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + // PointGet will use Fast Plan, so no Plan Cache, even for Key Partitioned tables. + tk.MustQuery(`select * from tk where a = 2`).Check(testkit.Rows("2 abc")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) +} + +func TestFixControl33031(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustQuery(`select @@session.tidb_enable_prepared_plan_cache`).Check(testkit.Rows("1")) + + tk.MustExec("use test") + tk.MustExec(`drop table if exists t`) + tk.MustExec(`CREATE TABLE t (a int primary key, b varchar(255), key (b)) PARTITION BY HASH (a) partitions 5`) + tk.MustExec(`insert into t values(0,0),(1,1),(2,2),(3,3),(4,4)`) + tk.MustExec(`insert into t select a + 5, b + 5 from t`) + tk.MustExec(`analyze table t`) + + tk.MustExec(`prepare stmt from 'select * from t where a = ?'`) + tk.MustExec(`set @a = 2`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("2 2")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`set @a = 3`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("3 3")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`set @@tidb_opt_fix_control = "33031:ON"`) + tk.MustExec(`set @a = 1`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("1 1")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 skip plan-cache: plan rebuild failed, Fix33031 fix-control set and partitioned table in cached Point Get plan")) + tk.MustExec(`set @@tidb_opt_fix_control = "33031:OFF"`) + tk.MustExec(`set @a = 2`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("2 2")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`deallocate prepare stmt`) + + tk.MustExec(`prepare stmt from 'select * from t where a IN (?,?)'`) + tk.MustExec(`set @a = 2, @b = 5`) + tk.MustQuery(`execute stmt using @a, @b`).Sort().Check(testkit.Rows("2 2", "5 5")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`set @a = 3, @b = 0`) + tk.MustQuery(`execute stmt using @a, @b`).Sort().Check(testkit.Rows("0 0", "3 3")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`set @@tidb_opt_fix_control = "33031:ON"`) + tk.MustExec(`set @a = 1, @b = 2`) + tk.MustQuery(`execute stmt using @a, @b`).Check(testkit.Rows("1 1", "2 2")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 skip plan-cache: plan rebuild failed, Fix33031 fix-control set and partitioned table in cached Batch Point Get plan")) + tk.MustExec(`set @@tidb_opt_fix_control = "33031:OFF"`) + tk.MustExec(`set @a = 2, @b = 3`) + tk.MustQuery(`execute stmt using @a, @b`).Check(testkit.Rows("2 2", "3 3")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) +} + +func TestPlanCachePartitionDuplicates(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t (a int unique key, b int) partition by range (a) ( + partition p0 values less than (10000), + partition p1 values less than (20000), + partition p2 values less than (30000), + partition p3 values less than (40000))`) + tk.MustExec(`insert into t values (1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7)`) + tk.MustExec(`insert into t select a + 10000, b + 10000 from t`) + tk.MustExec(`insert into t select a + 20000, b + 20000 from t`) + tk.MustExec(`analyze table t`) + tk.MustQuery(`select @@session.tidb_enable_prepared_plan_cache`).Check(testkit.Rows("1")) + tk.MustExec(`prepare stmt from 'select * from t use index(a) where a in (?,?,?)'`) + tk.MustExec(`set @a0 = 1, @a1 = 10001, @a2 = 2`) + tk.MustQuery(`execute stmt using @a0, @a1, @a2`).Sort().Check(testkit.Rows("1 1", "10001 10001", "2 2")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`set @a0 = 3, @a1 = 20001, @a2 = 50000`) + tk.MustQuery(`execute stmt using @a0, @a1, @a2`).Sort().Check(testkit.Rows("20001 20001", "3 3")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + 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)).CheckAt([]int{0}, [][]any{{"Batch_Point_Get_1"}}) + tk.MustExec(`set @a0 = 30003, @a1 = 20002, @a2 = 4`) + tk.MustQuery(`execute stmt using @a0, @a1, @a2`).Sort().Check(testkit.Rows("20002 20002", "30003 30003", "4 4")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tkExplain := testkit.NewTestKit(t, store) + tkExplain.MustExec(`use test`) + tkExplain.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).CheckAt([]int{0}, [][]any{{"Batch_Point_Get_1"}}) + tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 skip plan-cache: plan rebuild failed, rebuild to get an unsafe range, IndexValue length diff")) +} + +func TestPreparedStmtIndexLookup(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t (a int, b int, unique key (a)) +partition by hash (a) partitions 3`) + + for i := 0; i < 100; i++ { + tk.MustExec("insert into t values (?, ?)", i, i) + } + tk.MustExec("analyze table t") + tk.MustExec(`set tidb_partition_prune_mode = 'static'`) + tk.MustQuery(`select b from t where a = 1 or a = 10 or a = 10 or a = 999999`).Sort().Check(testkit.Rows("1", "10")) + tk.MustQuery(`explain format='brief' select b from t where a = 1 or a = 10 or a = 10 or a = 999999`).Check(testkit.Rows(""+ + "PartitionUnion 6.00 root ", + "├─Projection 3.00 root test.t.b", + "│ └─Batch_Point_Get 3.00 root table:t, partition:p0, index:a(a) keep order:false, desc:false", + "└─Projection 3.00 root test.t.b", + " └─Batch_Point_Get 3.00 root table:t, partition:p1, index:a(a) keep order:false, desc:false")) + tk.MustExec(`prepare stmt from 'select b from t where a = 1 or a = 10 or a = 10 or a = 999999'`) + tk.MustQuery(`execute stmt`) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + 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)).MultiCheckContain([]string{"PartitionUnion", "Batch_Point_Get", "partition:p0", "partition:p1"}) + tk.MustQuery(`execute stmt`) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 skip prepared plan-cache: query accesses partitioned tables is un-cacheable if tidb_partition_pruning_mode = 'static'")) +} + +func TestIndexRange(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + + tk.MustExec(`CREATE TABLE posts (id bigint NOT NULL AUTO_INCREMENT PRIMARY KEY)`) + tk.MustExec(`INSERT INTO posts (id) VALUES (1),(2),(3),(4),(5),(6),(7),(8),(9),(10),(11);`) + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1;`) + tk.MustQuery(`SELECT posts.* FROM posts WHERE (id = 1 or id = 9223372036854775808);`).Check(testkit.Rows("1")) +} diff --git a/tests/integrationtest/r/planner/core/partition_pruner.result b/tests/integrationtest/r/planner/core/partition_pruner.result new file mode 100644 index 0000000000000..b7048e6e2b11c --- /dev/null +++ b/tests/integrationtest/r/planner/core/partition_pruner.result @@ -0,0 +1,3258 @@ +set tidb_cost_model_version=2; +drop database if exists test_range_col_in; +create database test_range_col_in; +use test_range_col_in; +set @@session.tidb_partition_prune_mode='static'; +CREATE TABLE t1 ( +id bigint(20) NOT NULL AUTO_INCREMENT, +dt date, +PRIMARY KEY (id,dt) NONCLUSTERED) +PARTITION BY RANGE COLUMNS(dt) ( +PARTITION p20201125 VALUES LESS THAN ("20201126"), +PARTITION p20201126 VALUES LESS THAN ("20201127"), +PARTITION p20201127 VALUES LESS THAN ("20201128"), +PARTITION p20201128 VALUES LESS THAN ("20201129"), +PARTITION p20201129 VALUES LESS THAN ("20201130")); +explain format='brief' select /*+ HASH_AGG() */ count(1) from t1 where dt in ('2020-11-27','2020-11-28'); +id estRows task access object operator info +HashAgg 1.00 root funcs:count(Column#5)->Column#4 +└─PartitionUnion 2.00 root + ├─HashAgg 1.00 root funcs:count(Column#7)->Column#5 + │ └─IndexReader 1.00 root index:HashAgg + │ └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#7 + │ └─Selection 20.00 cop[tikv] in(test_range_col_in.t1.dt, 2020-11-27 00:00:00.000000, 2020-11-28 00:00:00.000000) + │ └─IndexFullScan 10000.00 cop[tikv] table:t1, partition:p20201127, index:PRIMARY(id, dt) keep order:false, stats:pseudo + └─HashAgg 1.00 root funcs:count(Column#10)->Column#5 + └─IndexReader 1.00 root index:HashAgg + └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#10 + └─Selection 20.00 cop[tikv] in(test_range_col_in.t1.dt, 2020-11-27 00:00:00.000000, 2020-11-28 00:00:00.000000) + └─IndexFullScan 10000.00 cop[tikv] table:t1, partition:p20201128, index:PRIMARY(id, dt) keep order:false, stats:pseudo +insert into t1 values (1, "2020-11-25"); +insert into t1 values (2, "2020-11-26"); +insert into t1 values (3, "2020-11-27"); +insert into t1 values (4, "2020-11-28"); +select id from t1 where dt in ('2020-11-27','2020-11-28') order by id; +id +3 +4 +select id from t1 where dt in (20201127,'2020-11-28') order by id; +id +3 +4 +select id from t1 where dt in (20201127,20201128) order by id; +id +3 +4 +select id from t1 where dt in (20201127,20201128,null) order by id; +id +3 +4 +select id from t1 where dt in ('2020-11-26','2020-11-25','2020-11-28') order by id; +id +1 +2 +4 +select id from t1 where dt in ('2020-11-26','wrong','2020-11-28') order by id; +id +2 +4 +create table t2 (a int) partition by range columns(a) ( +partition p0 values less than (0), +partition p1 values less than (10), +partition p2 values less than (20)); +insert into t2 values (-1), (1), (11), (null); +select a from t2 where a in (-1, 1) order by a; +a +-1 +1 +select a from t2 where a in (1, 11, null) order by a; +a +1 +11 +explain format='brief' select a from t2 where a in (-1, 1); +id estRows task access object operator info +PartitionUnion 40.00 root +├─TableReader 20.00 root data:Selection +│ └─Selection 20.00 cop[tikv] in(test_range_col_in.t2.a, -1, 1) +│ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p0 keep order:false, stats:pseudo +└─TableReader 20.00 root data:Selection + └─Selection 20.00 cop[tikv] in(test_range_col_in.t2.a, -1, 1) + └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo +create table t3 (a varchar(10)) partition by range columns(a) ( +partition p0 values less than ("aaa"), +partition p1 values less than ("bbb"), +partition p2 values less than ("ccc")); +explain format='brief' select a from t3 where a in ('aaa', 'aab'); +id estRows task access object operator info +TableReader 20.00 root data:Selection +└─Selection 20.00 cop[tikv] in(test_range_col_in.t3.a, "aaa", "aab") + └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo +explain format='brief' select a from t3 where a in ('aaa', 'bu'); +id estRows task access object operator info +PartitionUnion 40.00 root +├─TableReader 20.00 root data:Selection +│ └─Selection 20.00 cop[tikv] in(test_range_col_in.t3.a, "aaa", "bu") +│ └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p1 keep order:false, stats:pseudo +└─TableReader 20.00 root data:Selection + └─Selection 20.00 cop[tikv] in(test_range_col_in.t3.a, "aaa", "bu") + └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo +drop database if exists test_range_col_in_string; +create database test_range_col_in_string; +use test_range_col_in_string; +set names utf8mb4 collate utf8mb4_bin; +set @@session.tidb_partition_prune_mode='static'; +create table t (a varchar(255) charset utf8mb4 collate utf8mb4_bin) partition by range columns(a)( partition pNull values less than (""),partition pAAAA values less than ("AAAA"),partition pCCC values less than ("CCC"),partition pShrimpsandwich values less than ("Räksmörgås"),partition paaa values less than ("aaa"),partition pSushi values less than ("🍣🍣🍣"),partition pMax values less than (MAXVALUE)); +insert into t values (NULL), ("a"), ("Räkmacka"), ("🍣 is life"), ("🍺 after work?"), ("🍺🍺🍺🍺🍺 for oktoberfest"),("AA"),("aa"),("AAA"),("aaa"); +set @@tidb_partition_prune_mode = 'dynamic'; +explain format = 'brief' select * from t where a IS NULL; +id estRows task access object operator info +TableReader 10.00 root partition:pNull data:Selection +└─Selection 10.00 cop[tikv] isnull(test_range_col_in_string.t.a) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a IS NULL; +a +NULL +explain format = 'brief' select * from t where a = 'AA'; +id estRows task access object operator info +TableReader 10.00 root partition:pAAAA data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AA'; +a +AA +explain format = 'brief' select * from t where a = 'AA' collate utf8mb4_general_ci; +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AA' collate utf8mb4_general_ci; +a +AA +aa +explain format = 'brief' select * from t where a = 'aa'; +id estRows task access object operator info +TableReader 10.00 root partition:paaa data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'aa'; +a +aa +explain format = 'brief' select * from t where a = 'aa' collate utf8mb4_general_ci; +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'aa' collate utf8mb4_general_ci; +a +AA +aa +explain format = 'brief' select * from t where a collate utf8mb4_general_ci = 'aa'; +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a collate utf8mb4_general_ci = 'aa'; +a +AA +aa +explain format = 'brief' select * from t where a = 'AAA'; +id estRows task access object operator info +TableReader 10.00 root partition:pAAAA data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AAA'; +a +AAA +explain format = 'brief' select * from t where a = 'AB'; +id estRows task access object operator info +TableReader 10.00 root partition:pCCC data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AB") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AB'; +a +explain format = 'brief' select * from t where a = 'aB'; +id estRows task access object operator info +TableReader 10.00 root partition:paaa data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aB") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'aB'; +a +explain format = 'brief' select * from t where a = '🍣'; +id estRows task access object operator info +TableReader 10.00 root partition:pSushi data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "🍣") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = '🍣'; +a +explain format = 'brief' select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +id estRows task access object operator info +TableReader 30.00 root partition:pShrimpsandwich,pSushi,pMax data:Selection +└─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +a +Räkmacka +🍣 is life +explain format = 'brief' select * from t where a in ('AAA', 'aa'); +id estRows task access object operator info +TableReader 20.00 root partition:pAAAA,paaa data:Selection +└─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa'); +a +AAA +aa +explain format = 'brief' select * from t where a in ('AAA' collate utf8mb4_general_ci, 'aa'); +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA' collate utf8mb4_general_ci, 'aa'); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a in ('AAA', 'aa' collate utf8mb4_general_ci); +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa' collate utf8mb4_general_ci); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a collate utf8mb4_general_ci in ('AAA', 'aa'); +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a collate utf8mb4_general_ci in ('AAA', 'aa'); +a +AA +AAA +aa +aaa +set @@tidb_partition_prune_mode = 'static'; +explain format = 'brief' select * from t where a IS NULL; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] isnull(test_range_col_in_string.t.a) + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +select * from t where a IS NULL; +a +NULL +explain format = 'brief' select * from t where a = 'AA'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a = 'AA'; +a +AA +explain format = 'brief' select * from t where a = 'AA' collate utf8mb4_general_ci; +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a = 'AA' collate utf8mb4_general_ci; +a +AA +aa +explain format = 'brief' select * from t where a = 'aa'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a = 'aa'; +a +aa +explain format = 'brief' select * from t where a = 'aa' collate utf8mb4_general_ci; +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a = 'aa' collate utf8mb4_general_ci; +a +AA +aa +explain format = 'brief' select * from t where a collate utf8mb4_general_ci = 'aa'; +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a collate utf8mb4_general_ci = 'aa'; +a +AA +aa +explain format = 'brief' select * from t where a = 'AAA'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a = 'AAA'; +a +AAA +explain format = 'brief' select * from t where a = 'AB'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AB") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +select * from t where a = 'AB'; +a +explain format = 'brief' select * from t where a = 'aB'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aB") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a = 'aB'; +a +explain format = 'brief' select * from t where a = '🍣'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "🍣") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +select * from t where a = '🍣'; +a +explain format = 'brief' select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +id estRows task access object operator info +PartitionUnion 90.00 root +├─TableReader 30.00 root data:Selection +│ └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 30.00 root data:Selection +│ └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 30.00 root data:Selection + └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +a +Räkmacka +🍣 is life +explain format = 'brief' select * from t where a in ('AAA', 'aa'); +id estRows task access object operator info +PartitionUnion 40.00 root +├─TableReader 20.00 root data:Selection +│ └─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +└─TableReader 20.00 root data:Selection + └─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa'); +a +AAA +aa +explain format = 'brief' select * from t where a in ('AAA' collate utf8mb4_general_ci, 'aa'); +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a in ('AAA' collate utf8mb4_general_ci, 'aa'); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a in ('AAA', 'aa' collate utf8mb4_general_ci); +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa' collate utf8mb4_general_ci); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a collate utf8mb4_general_ci in ('AAA', 'aa'); +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a collate utf8mb4_general_ci in ('AAA', 'aa'); +a +AA +AAA +aa +aaa +set names utf8mb4 collate utf8mb4_general_ci; +set @@tidb_partition_prune_mode = 'dynamic'; +explain format = 'brief' select * from t where a IS NULL; +id estRows task access object operator info +TableReader 10.00 root partition:pNull data:Selection +└─Selection 10.00 cop[tikv] isnull(test_range_col_in_string.t.a) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a IS NULL; +a +NULL +explain format = 'brief' select * from t where a = 'AA'; +id estRows task access object operator info +TableReader 10.00 root partition:pAAAA data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AA'; +a +AA +explain format = 'brief' select * from t where a = 'AA' collate utf8mb4_general_ci; +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AA' collate utf8mb4_general_ci; +a +AA +aa +explain format = 'brief' select * from t where a = 'aa'; +id estRows task access object operator info +TableReader 10.00 root partition:paaa data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'aa'; +a +aa +explain format = 'brief' select * from t where a = 'aa' collate utf8mb4_general_ci; +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'aa' collate utf8mb4_general_ci; +a +AA +aa +explain format = 'brief' select * from t where a collate utf8mb4_general_ci = 'aa'; +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a collate utf8mb4_general_ci = 'aa'; +a +AA +aa +explain format = 'brief' select * from t where a = 'AAA'; +id estRows task access object operator info +TableReader 10.00 root partition:pAAAA data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AAA'; +a +AAA +explain format = 'brief' select * from t where a = 'AB'; +id estRows task access object operator info +TableReader 10.00 root partition:pCCC data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AB") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AB'; +a +explain format = 'brief' select * from t where a = 'aB'; +id estRows task access object operator info +TableReader 10.00 root partition:paaa data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aB") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'aB'; +a +explain format = 'brief' select * from t where a = '🍣'; +id estRows task access object operator info +TableReader 10.00 root partition:pSushi data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "🍣") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = '🍣'; +a +explain format = 'brief' select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +id estRows task access object operator info +TableReader 30.00 root partition:pShrimpsandwich,pSushi,pMax data:Selection +└─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +a +Räkmacka +🍣 is life +explain format = 'brief' select * from t where a in ('AAA', 'aa'); +id estRows task access object operator info +TableReader 20.00 root partition:pAAAA,paaa data:Selection +└─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa'); +a +AAA +aa +explain format = 'brief' select * from t where a in ('AAA' collate utf8mb4_general_ci, 'aa'); +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA' collate utf8mb4_general_ci, 'aa'); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a in ('AAA', 'aa' collate utf8mb4_general_ci); +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa' collate utf8mb4_general_ci); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a collate utf8mb4_general_ci in ('AAA', 'aa'); +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a collate utf8mb4_general_ci in ('AAA', 'aa'); +a +AA +AAA +aa +aaa +set @@tidb_partition_prune_mode = 'static'; +explain format = 'brief' select * from t where a IS NULL; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] isnull(test_range_col_in_string.t.a) + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +select * from t where a IS NULL; +a +NULL +explain format = 'brief' select * from t where a = 'AA'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a = 'AA'; +a +AA +explain format = 'brief' select * from t where a = 'AA' collate utf8mb4_general_ci; +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a = 'AA' collate utf8mb4_general_ci; +a +AA +aa +explain format = 'brief' select * from t where a = 'aa'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a = 'aa'; +a +aa +explain format = 'brief' select * from t where a = 'aa' collate utf8mb4_general_ci; +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a = 'aa' collate utf8mb4_general_ci; +a +AA +aa +explain format = 'brief' select * from t where a collate utf8mb4_general_ci = 'aa'; +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a collate utf8mb4_general_ci = 'aa'; +a +AA +aa +explain format = 'brief' select * from t where a = 'AAA'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a = 'AAA'; +a +AAA +explain format = 'brief' select * from t where a = 'AB'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AB") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +select * from t where a = 'AB'; +a +explain format = 'brief' select * from t where a = 'aB'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aB") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a = 'aB'; +a +explain format = 'brief' select * from t where a = '🍣'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "🍣") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +select * from t where a = '🍣'; +a +explain format = 'brief' select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +id estRows task access object operator info +PartitionUnion 90.00 root +├─TableReader 30.00 root data:Selection +│ └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 30.00 root data:Selection +│ └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 30.00 root data:Selection + └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +a +Räkmacka +🍣 is life +explain format = 'brief' select * from t where a in ('AAA', 'aa'); +id estRows task access object operator info +PartitionUnion 40.00 root +├─TableReader 20.00 root data:Selection +│ └─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +└─TableReader 20.00 root data:Selection + └─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa'); +a +AAA +aa +explain format = 'brief' select * from t where a in ('AAA' collate utf8mb4_general_ci, 'aa'); +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a in ('AAA' collate utf8mb4_general_ci, 'aa'); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a in ('AAA', 'aa' collate utf8mb4_general_ci); +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa' collate utf8mb4_general_ci); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a collate utf8mb4_general_ci in ('AAA', 'aa'); +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a collate utf8mb4_general_ci in ('AAA', 'aa'); +a +AA +AAA +aa +aaa +set names utf8mb4 collate utf8mb4_unicode_ci; +set @@tidb_partition_prune_mode = 'dynamic'; +explain format = 'brief' select * from t where a IS NULL; +id estRows task access object operator info +TableReader 10.00 root partition:pNull data:Selection +└─Selection 10.00 cop[tikv] isnull(test_range_col_in_string.t.a) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a IS NULL; +a +NULL +explain format = 'brief' select * from t where a = 'AA'; +id estRows task access object operator info +TableReader 10.00 root partition:pAAAA data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AA'; +a +AA +explain format = 'brief' select * from t where a = 'AA' collate utf8mb4_general_ci; +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AA' collate utf8mb4_general_ci; +a +AA +aa +explain format = 'brief' select * from t where a = 'aa'; +id estRows task access object operator info +TableReader 10.00 root partition:paaa data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'aa'; +a +aa +explain format = 'brief' select * from t where a = 'aa' collate utf8mb4_general_ci; +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'aa' collate utf8mb4_general_ci; +a +AA +aa +explain format = 'brief' select * from t where a collate utf8mb4_general_ci = 'aa'; +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a collate utf8mb4_general_ci = 'aa'; +a +AA +aa +explain format = 'brief' select * from t where a = 'AAA'; +id estRows task access object operator info +TableReader 10.00 root partition:pAAAA data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AAA'; +a +AAA +explain format = 'brief' select * from t where a = 'AB'; +id estRows task access object operator info +TableReader 10.00 root partition:pCCC data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AB") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AB'; +a +explain format = 'brief' select * from t where a = 'aB'; +id estRows task access object operator info +TableReader 10.00 root partition:paaa data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aB") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'aB'; +a +explain format = 'brief' select * from t where a = '🍣'; +id estRows task access object operator info +TableReader 10.00 root partition:pSushi data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "🍣") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = '🍣'; +a +explain format = 'brief' select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +id estRows task access object operator info +TableReader 30.00 root partition:pShrimpsandwich,pSushi,pMax data:Selection +└─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +a +Räkmacka +🍣 is life +explain format = 'brief' select * from t where a in ('AAA', 'aa'); +id estRows task access object operator info +TableReader 20.00 root partition:pAAAA,paaa data:Selection +└─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa'); +a +AAA +aa +explain format = 'brief' select * from t where a in ('AAA' collate utf8mb4_general_ci, 'aa'); +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA' collate utf8mb4_general_ci, 'aa'); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a in ('AAA', 'aa' collate utf8mb4_general_ci); +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa' collate utf8mb4_general_ci); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a collate utf8mb4_general_ci in ('AAA', 'aa'); +id estRows task access object operator info +TableReader 8000.00 root partition:all data:Selection +└─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a collate utf8mb4_general_ci in ('AAA', 'aa'); +a +AA +AAA +aa +aaa +set @@tidb_partition_prune_mode = 'static'; +explain format = 'brief' select * from t where a IS NULL; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] isnull(test_range_col_in_string.t.a) + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +select * from t where a IS NULL; +a +NULL +explain format = 'brief' select * from t where a = 'AA'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a = 'AA'; +a +AA +explain format = 'brief' select * from t where a = 'AA' collate utf8mb4_general_ci; +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a = 'AA' collate utf8mb4_general_ci; +a +AA +aa +explain format = 'brief' select * from t where a = 'aa'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a = 'aa'; +a +aa +explain format = 'brief' select * from t where a = 'aa' collate utf8mb4_general_ci; +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a = 'aa' collate utf8mb4_general_ci; +a +AA +aa +explain format = 'brief' select * from t where a collate utf8mb4_general_ci = 'aa'; +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] eq(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a collate utf8mb4_general_ci = 'aa'; +a +AA +aa +explain format = 'brief' select * from t where a = 'AAA'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a = 'AAA'; +a +AAA +explain format = 'brief' select * from t where a = 'AB'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AB") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +select * from t where a = 'AB'; +a +explain format = 'brief' select * from t where a = 'aB'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aB") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a = 'aB'; +a +explain format = 'brief' select * from t where a = '🍣'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "🍣") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +select * from t where a = '🍣'; +a +explain format = 'brief' select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +id estRows task access object operator info +PartitionUnion 90.00 root +├─TableReader 30.00 root data:Selection +│ └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 30.00 root data:Selection +│ └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 30.00 root data:Selection + └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +a +Räkmacka +🍣 is life +explain format = 'brief' select * from t where a in ('AAA', 'aa'); +id estRows task access object operator info +PartitionUnion 40.00 root +├─TableReader 20.00 root data:Selection +│ └─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +└─TableReader 20.00 root data:Selection + └─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa'); +a +AAA +aa +explain format = 'brief' select * from t where a in ('AAA' collate utf8mb4_general_ci, 'aa'); +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a in ('AAA' collate utf8mb4_general_ci, 'aa'); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a in ('AAA', 'aa' collate utf8mb4_general_ci); +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa' collate utf8mb4_general_ci); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a collate utf8mb4_general_ci in ('AAA', 'aa'); +id estRows task access object operator info +PartitionUnion 56000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(cast(test_range_col_in_string.t.a, varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci), "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a collate utf8mb4_general_ci in ('AAA', 'aa'); +a +AA +AAA +aa +aaa +drop table t; +create table t (a varchar(255) charset utf8mb4 collate utf8mb4_general_ci) partition by range columns(a)( partition pNull values less than (""),partition paaa values less than ("aaa"),partition pAAAA values less than ("AAAA"),partition pCCC values less than ("CCC"),partition pShrimpsandwich values less than ("Räksmörgås"),partition pSushi values less than ("🍣🍣🍣"),partition pMax values less than (MAXVALUE)); +insert into t values (NULL), ("a"), ("Räkmacka"), ("🍣 is life"), ("🍺 after work?"), ("🍺🍺🍺🍺🍺 for oktoberfest"),("AA"),("aa"),("AAA"),("aaa"); +set names utf8mb4 collate utf8mb4_bin; +set @@tidb_partition_prune_mode = 'dynamic'; +explain format = 'brief' select * from t where a IS NULL; +id estRows task access object operator info +TableReader 10.00 root partition:pNull data:Selection +└─Selection 10.00 cop[tikv] isnull(test_range_col_in_string.t.a) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a IS NULL; +a +NULL +explain format = 'brief' select * from t where a = 'AA'; +id estRows task access object operator info +TableReader 10.00 root partition:paaa data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AA'; +a +AA +aa +explain format = 'brief' select * from t where a = 'AA' collate utf8mb4_bin; +id estRows task access object operator info +TableReader 8000.00 root partition:paaa data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AA' collate utf8mb4_bin; +a +AA +explain format = 'brief' select * from t where a = 'AAA'; +id estRows task access object operator info +TableReader 10.00 root partition:pAAAA data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AAA'; +a +AAA +aaa +explain format = 'brief' select * from t where a = 'AAA' collate utf8mb4_bin; +id estRows task access object operator info +TableReader 8000.00 root partition:pAAAA data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AAA' collate utf8mb4_bin; +a +AAA +explain format = 'brief' select * from t where a = 'AB'; +id estRows task access object operator info +TableReader 10.00 root partition:pCCC data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AB") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AB'; +a +explain format = 'brief' select * from t where a = 'aB'; +id estRows task access object operator info +TableReader 10.00 root partition:pCCC data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aB") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'aB'; +a +explain format = 'brief' select * from t where a = '🍣'; +id estRows task access object operator info +TableReader 10.00 root partition:pSushi data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "🍣") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = '🍣'; +a +explain format = 'brief' select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +id estRows task access object operator info +TableReader 30.00 root partition:pShrimpsandwich,pSushi,pMax data:Selection +└─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +a +Räkmacka +🍣 is life +explain format = 'brief' select * from t where a in ('AA', 'aaa'); +id estRows task access object operator info +TableReader 20.00 root partition:paaa,pAAAA data:Selection +└─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AA", "aaa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AA', 'aaa'); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a in ('AAA' collate utf8mb4_bin, 'aa'); +id estRows task access object operator info +TableReader 8000.00 root partition:paaa,pAAAA data:Selection +└─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA' collate utf8mb4_bin, 'aa'); +a +AAA +aa +explain format = 'brief' select * from t where a in ('AAA', 'aa' collate utf8mb4_bin); +id estRows task access object operator info +TableReader 8000.00 root partition:paaa,pAAAA data:Selection +└─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa' collate utf8mb4_bin); +a +AAA +aa +set @@tidb_partition_prune_mode = 'static'; +explain format = 'brief' select * from t where a IS NULL; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] isnull(test_range_col_in_string.t.a) + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +select * from t where a IS NULL; +a +NULL +explain format = 'brief' select * from t where a = 'AA'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a = 'AA'; +a +AA +aa +explain format = 'brief' select * from t where a = 'AA' collate utf8mb4_bin; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a = 'AA' collate utf8mb4_bin; +a +AA +explain format = 'brief' select * from t where a = 'AAA'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a = 'AAA'; +a +AAA +aaa +explain format = 'brief' select * from t where a = 'AAA' collate utf8mb4_bin; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a = 'AAA' collate utf8mb4_bin; +a +AAA +explain format = 'brief' select * from t where a = 'AB'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AB") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +select * from t where a = 'AB'; +a +explain format = 'brief' select * from t where a = 'aB'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aB") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +select * from t where a = 'aB'; +a +explain format = 'brief' select * from t where a = '🍣'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "🍣") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +select * from t where a = '🍣'; +a +explain format = 'brief' select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +id estRows task access object operator info +PartitionUnion 90.00 root +├─TableReader 30.00 root data:Selection +│ └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 30.00 root data:Selection +│ └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 30.00 root data:Selection + └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +a +Räkmacka +🍣 is life +explain format = 'brief' select * from t where a in ('AA', 'aaa'); +id estRows task access object operator info +PartitionUnion 40.00 root +├─TableReader 20.00 root data:Selection +│ └─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AA", "aaa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +└─TableReader 20.00 root data:Selection + └─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AA", "aaa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a in ('AA', 'aaa'); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a in ('AAA' collate utf8mb4_bin, 'aa'); +id estRows task access object operator info +PartitionUnion 16000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a in ('AAA' collate utf8mb4_bin, 'aa'); +a +AAA +aa +explain format = 'brief' select * from t where a in ('AAA', 'aa' collate utf8mb4_bin); +id estRows task access object operator info +PartitionUnion 16000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa' collate utf8mb4_bin); +a +AAA +aa +set names utf8mb4 collate utf8mb4_general_ci; +set @@tidb_partition_prune_mode = 'dynamic'; +explain format = 'brief' select * from t where a IS NULL; +id estRows task access object operator info +TableReader 10.00 root partition:pNull data:Selection +└─Selection 10.00 cop[tikv] isnull(test_range_col_in_string.t.a) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a IS NULL; +a +NULL +explain format = 'brief' select * from t where a = 'AA'; +id estRows task access object operator info +TableReader 10.00 root partition:paaa data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AA'; +a +AA +aa +explain format = 'brief' select * from t where a = 'AA' collate utf8mb4_bin; +id estRows task access object operator info +TableReader 8000.00 root partition:paaa data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AA' collate utf8mb4_bin; +a +AA +explain format = 'brief' select * from t where a = 'AAA'; +id estRows task access object operator info +TableReader 10.00 root partition:pAAAA data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AAA'; +a +AAA +aaa +explain format = 'brief' select * from t where a = 'AAA' collate utf8mb4_bin; +id estRows task access object operator info +TableReader 8000.00 root partition:pAAAA data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AAA' collate utf8mb4_bin; +a +AAA +explain format = 'brief' select * from t where a = 'AB'; +id estRows task access object operator info +TableReader 10.00 root partition:pCCC data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AB") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AB'; +a +explain format = 'brief' select * from t where a = 'aB'; +id estRows task access object operator info +TableReader 10.00 root partition:pCCC data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aB") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'aB'; +a +explain format = 'brief' select * from t where a = '🍣'; +id estRows task access object operator info +TableReader 10.00 root partition:pSushi data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "🍣") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = '🍣'; +a +explain format = 'brief' select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +id estRows task access object operator info +TableReader 30.00 root partition:pShrimpsandwich,pSushi,pMax data:Selection +└─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +a +Räkmacka +🍣 is life +explain format = 'brief' select * from t where a in ('AA', 'aaa'); +id estRows task access object operator info +TableReader 20.00 root partition:paaa,pAAAA data:Selection +└─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AA", "aaa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AA', 'aaa'); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a in ('AAA' collate utf8mb4_bin, 'aa'); +id estRows task access object operator info +TableReader 8000.00 root partition:paaa,pAAAA data:Selection +└─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA' collate utf8mb4_bin, 'aa'); +a +AAA +aa +explain format = 'brief' select * from t where a in ('AAA', 'aa' collate utf8mb4_bin); +id estRows task access object operator info +TableReader 8000.00 root partition:paaa,pAAAA data:Selection +└─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa' collate utf8mb4_bin); +a +AAA +aa +set @@tidb_partition_prune_mode = 'static'; +explain format = 'brief' select * from t where a IS NULL; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] isnull(test_range_col_in_string.t.a) + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +select * from t where a IS NULL; +a +NULL +explain format = 'brief' select * from t where a = 'AA'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a = 'AA'; +a +AA +aa +explain format = 'brief' select * from t where a = 'AA' collate utf8mb4_bin; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a = 'AA' collate utf8mb4_bin; +a +AA +explain format = 'brief' select * from t where a = 'AAA'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a = 'AAA'; +a +AAA +aaa +explain format = 'brief' select * from t where a = 'AAA' collate utf8mb4_bin; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a = 'AAA' collate utf8mb4_bin; +a +AAA +explain format = 'brief' select * from t where a = 'AB'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AB") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +select * from t where a = 'AB'; +a +explain format = 'brief' select * from t where a = 'aB'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aB") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +select * from t where a = 'aB'; +a +explain format = 'brief' select * from t where a = '🍣'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "🍣") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +select * from t where a = '🍣'; +a +explain format = 'brief' select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +id estRows task access object operator info +PartitionUnion 90.00 root +├─TableReader 30.00 root data:Selection +│ └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 30.00 root data:Selection +│ └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 30.00 root data:Selection + └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +a +Räkmacka +🍣 is life +explain format = 'brief' select * from t where a in ('AA', 'aaa'); +id estRows task access object operator info +PartitionUnion 40.00 root +├─TableReader 20.00 root data:Selection +│ └─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AA", "aaa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +└─TableReader 20.00 root data:Selection + └─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AA", "aaa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a in ('AA', 'aaa'); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a in ('AAA' collate utf8mb4_bin, 'aa'); +id estRows task access object operator info +PartitionUnion 16000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a in ('AAA' collate utf8mb4_bin, 'aa'); +a +AAA +aa +explain format = 'brief' select * from t where a in ('AAA', 'aa' collate utf8mb4_bin); +id estRows task access object operator info +PartitionUnion 16000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa' collate utf8mb4_bin); +a +AAA +aa +set names utf8mb4 collate utf8mb4_unicode_ci; +set @@tidb_partition_prune_mode = 'dynamic'; +explain format = 'brief' select * from t where a IS NULL; +id estRows task access object operator info +TableReader 10.00 root partition:pNull data:Selection +└─Selection 10.00 cop[tikv] isnull(test_range_col_in_string.t.a) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a IS NULL; +a +NULL +explain format = 'brief' select * from t where a = 'AA'; +id estRows task access object operator info +TableReader 10.00 root partition:paaa data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AA'; +a +AA +aa +explain format = 'brief' select * from t where a = 'AA' collate utf8mb4_bin; +id estRows task access object operator info +TableReader 8000.00 root partition:paaa data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AA' collate utf8mb4_bin; +a +AA +explain format = 'brief' select * from t where a = 'AAA'; +id estRows task access object operator info +TableReader 10.00 root partition:pAAAA data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AAA'; +a +AAA +aaa +explain format = 'brief' select * from t where a = 'AAA' collate utf8mb4_bin; +id estRows task access object operator info +TableReader 8000.00 root partition:pAAAA data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AAA' collate utf8mb4_bin; +a +AAA +explain format = 'brief' select * from t where a = 'AB'; +id estRows task access object operator info +TableReader 10.00 root partition:pCCC data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AB") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'AB'; +a +explain format = 'brief' select * from t where a = 'aB'; +id estRows task access object operator info +TableReader 10.00 root partition:pCCC data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aB") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = 'aB'; +a +explain format = 'brief' select * from t where a = '🍣'; +id estRows task access object operator info +TableReader 10.00 root partition:pSushi data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "🍣") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a = '🍣'; +a +explain format = 'brief' select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +id estRows task access object operator info +TableReader 30.00 root partition:pShrimpsandwich,pSushi,pMax data:Selection +└─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +a +Räkmacka +🍣 is life +explain format = 'brief' select * from t where a in ('AA', 'aaa'); +id estRows task access object operator info +TableReader 20.00 root partition:paaa,pAAAA data:Selection +└─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AA", "aaa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AA', 'aaa'); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a in ('AAA' collate utf8mb4_bin, 'aa'); +id estRows task access object operator info +TableReader 8000.00 root partition:paaa,pAAAA data:Selection +└─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA' collate utf8mb4_bin, 'aa'); +a +AAA +aa +explain format = 'brief' select * from t where a in ('AAA', 'aa' collate utf8mb4_bin); +id estRows task access object operator info +TableReader 8000.00 root partition:paaa,pAAAA data:Selection +└─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa' collate utf8mb4_bin); +a +AAA +aa +set @@tidb_partition_prune_mode = 'static'; +explain format = 'brief' select * from t where a IS NULL; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] isnull(test_range_col_in_string.t.a) + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pNull keep order:false, stats:pseudo +select * from t where a IS NULL; +a +NULL +explain format = 'brief' select * from t where a = 'AA'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a = 'AA'; +a +AA +aa +explain format = 'brief' select * from t where a = 'AA' collate utf8mb4_bin; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +select * from t where a = 'AA' collate utf8mb4_bin; +a +AA +explain format = 'brief' select * from t where a = 'AAA'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a = 'AAA'; +a +AAA +aaa +explain format = 'brief' select * from t where a = 'AAA' collate utf8mb4_bin; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] eq(test_range_col_in_string.t.a, "AAA") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a = 'AAA' collate utf8mb4_bin; +a +AAA +explain format = 'brief' select * from t where a = 'AB'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "AB") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +select * from t where a = 'AB'; +a +explain format = 'brief' select * from t where a = 'aB'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "aB") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pCCC keep order:false, stats:pseudo +select * from t where a = 'aB'; +a +explain format = 'brief' select * from t where a = '🍣'; +id estRows task access object operator info +TableReader 10.00 root data:Selection +└─Selection 10.00 cop[tikv] eq(test_range_col_in_string.t.a, "🍣") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +select * from t where a = '🍣'; +a +explain format = 'brief' select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +id estRows task access object operator info +PartitionUnion 90.00 root +├─TableReader 30.00 root data:Selection +│ └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pShrimpsandwich keep order:false, stats:pseudo +├─TableReader 30.00 root data:Selection +│ └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:pSushi keep order:false, stats:pseudo +└─TableReader 30.00 root data:Selection + └─Selection 30.00 cop[tikv] in(test_range_col_in_string.t.a, "🍣 is life", "Räkmacka", "🍺🍺🍺🍺 after work?") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pMax keep order:false, stats:pseudo +select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?"); +a +Räkmacka +🍣 is life +explain format = 'brief' select * from t where a in ('AA', 'aaa'); +id estRows task access object operator info +PartitionUnion 40.00 root +├─TableReader 20.00 root data:Selection +│ └─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AA", "aaa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +└─TableReader 20.00 root data:Selection + └─Selection 20.00 cop[tikv] in(test_range_col_in_string.t.a, "AA", "aaa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a in ('AA', 'aaa'); +a +AA +AAA +aa +aaa +explain format = 'brief' select * from t where a in ('AAA' collate utf8mb4_bin, 'aa'); +id estRows task access object operator info +PartitionUnion 16000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a in ('AAA' collate utf8mb4_bin, 'aa'); +a +AAA +aa +explain format = 'brief' select * from t where a in ('AAA', 'aa' collate utf8mb4_bin); +id estRows task access object operator info +PartitionUnion 16000.00 root +├─TableReader 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") +│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:paaa keep order:false, stats:pseudo +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] in(test_range_col_in_string.t.a, "AAA", "aa") + └─TableFullScan 10000.00 cop[tikv] table:t, partition:pAAAA keep order:false, stats:pseudo +select * from t where a in ('AAA', 'aa' collate utf8mb4_bin); +a +AAA +aa +drop database if exists test_partition; +create database test_partition; +use test_partition; +set @@session.tidb_enable_list_partition = ON; +create table t1 (id int, a int, b int ) partition by list columns (b, id, a) (partition p0 values in ((1,0,2),(2,0,2),(0,1,0),(1,1,0),(2,1,0),(0,1,1),(0,1,2),(0,2,0),(1,2,0)),partition p1 values in ((1,0,1),(0,0,2),(2,1,1),(2,1,2),(2,2,1),(1,2,2),(2,2,2)),partition p2 values in ((0,0,0),(1,0,0),(2,0,0),(0,0,1),(2,0,1),(1,1,1),(1,1,2),(2,2,0),(0,2,1),(1,2,1),(0,2,2))); +drop database if exists test_partition_1; +create database test_partition_1; +use test_partition_1; +create table t1 (id int, a int, b int); +insert into t1 (b,id,a) values (1,0,2),(2,0,2),(0,1,0),(1,1,0),(2,1,0),(0,1,1),(0,1,2),(0,2,0),(1,2,0); +insert into t1 (b,id,a) values (1,0,2),(2,0,2),(0,1,0),(1,1,0),(2,1,0),(0,1,1),(0,1,2),(0,2,0),(1,2,0); +select * from t1 order by id,a,b; +id a b +0 2 1 +0 2 1 +0 2 2 +0 2 2 +1 0 0 +1 0 0 +1 0 1 +1 0 1 +1 0 2 +1 0 2 +1 1 0 +1 1 0 +1 2 0 +1 2 0 +2 0 0 +2 0 0 +2 0 1 +2 0 1 +select * from t1 order by id,a,b; +id a b +0 2 1 +0 2 1 +0 2 2 +0 2 2 +1 0 0 +1 0 0 +1 0 1 +1 0 1 +1 0 2 +1 0 2 +1 1 0 +1 1 0 +1 2 0 +1 2 0 +2 0 0 +2 0 0 +2 0 1 +2 0 1 +insert into t1 (b,id,a) values (1,0,1),(0,0,2),(2,1,1),(2,1,2),(2,2,1),(1,2,2),(2,2,2); +insert into t1 (b,id,a) values (1,0,1),(0,0,2),(2,1,1),(2,1,2),(2,2,1),(1,2,2),(2,2,2); +select * from t1 order by id,a,b; +id a b +0 1 1 +0 1 1 +0 2 0 +0 2 0 +0 2 1 +0 2 1 +0 2 2 +0 2 2 +1 0 0 +1 0 0 +1 0 1 +1 0 1 +1 0 2 +1 0 2 +1 1 0 +1 1 0 +1 1 2 +1 1 2 +1 2 0 +1 2 0 +1 2 2 +1 2 2 +2 0 0 +2 0 0 +2 0 1 +2 0 1 +2 1 2 +2 1 2 +2 2 1 +2 2 1 +2 2 2 +2 2 2 +select * from t1 order by id,a,b; +id a b +0 1 1 +0 1 1 +0 2 0 +0 2 0 +0 2 1 +0 2 1 +0 2 2 +0 2 2 +1 0 0 +1 0 0 +1 0 1 +1 0 1 +1 0 2 +1 0 2 +1 1 0 +1 1 0 +1 1 2 +1 1 2 +1 2 0 +1 2 0 +1 2 2 +1 2 2 +2 0 0 +2 0 0 +2 0 1 +2 0 1 +2 1 2 +2 1 2 +2 2 1 +2 2 1 +2 2 2 +2 2 2 +insert into t1 (b,id,a) values (0,0,0),(1,0,0),(2,0,0),(0,0,1),(2,0,1),(1,1,1),(1,1,2),(2,2,0),(0,2,1),(1,2,1),(0,2,2); +insert into t1 (b,id,a) values (0,0,0),(1,0,0),(2,0,0),(0,0,1),(2,0,1),(1,1,1),(1,1,2),(2,2,0),(0,2,1),(1,2,1),(0,2,2); +select * from t1 order by id,a,b; +id a b +0 0 0 +0 0 0 +0 0 1 +0 0 1 +0 0 2 +0 0 2 +0 1 0 +0 1 0 +0 1 1 +0 1 1 +0 1 2 +0 1 2 +0 2 0 +0 2 0 +0 2 1 +0 2 1 +0 2 2 +0 2 2 +1 0 0 +1 0 0 +1 0 1 +1 0 1 +1 0 2 +1 0 2 +1 1 0 +1 1 0 +1 1 1 +1 1 1 +1 1 2 +1 1 2 +1 2 0 +1 2 0 +1 2 1 +1 2 1 +1 2 2 +1 2 2 +2 0 0 +2 0 0 +2 0 1 +2 0 1 +2 0 2 +2 0 2 +2 1 0 +2 1 0 +2 1 1 +2 1 1 +2 1 2 +2 1 2 +2 2 0 +2 2 0 +2 2 1 +2 2 1 +2 2 2 +2 2 2 +select * from t1 order by id,a,b; +id a b +0 0 0 +0 0 0 +0 0 1 +0 0 1 +0 0 2 +0 0 2 +0 1 0 +0 1 0 +0 1 1 +0 1 1 +0 1 2 +0 1 2 +0 2 0 +0 2 0 +0 2 1 +0 2 1 +0 2 2 +0 2 2 +1 0 0 +1 0 0 +1 0 1 +1 0 1 +1 0 2 +1 0 2 +1 1 0 +1 1 0 +1 1 1 +1 1 1 +1 1 2 +1 1 2 +1 2 0 +1 2 0 +1 2 1 +1 2 1 +1 2 2 +1 2 2 +2 0 0 +2 0 0 +2 0 1 +2 0 1 +2 0 2 +2 0 2 +2 1 0 +2 1 0 +2 1 1 +2 1 1 +2 1 2 +2 1 2 +2 2 0 +2 2 0 +2 2 1 +2 2 1 +2 2 2 +2 2 2 +select * from t1 where id = 0 order by id,a,b; +id a b +0 0 0 +0 0 0 +0 0 1 +0 0 1 +0 0 2 +0 0 2 +0 1 0 +0 1 0 +0 1 1 +0 1 1 +0 1 2 +0 1 2 +0 2 0 +0 2 0 +0 2 1 +0 2 1 +0 2 2 +0 2 2 +select * from t1 where id = 0 order by id,a,b; +id a b +0 0 0 +0 0 0 +0 0 1 +0 0 1 +0 0 2 +0 0 2 +0 1 0 +0 1 0 +0 1 1 +0 1 1 +0 1 2 +0 1 2 +0 2 0 +0 2 0 +0 2 1 +0 2 1 +0 2 2 +0 2 2 +select * from t1 where a = 0 order by id,a,b; +id a b +0 0 0 +0 0 0 +0 0 1 +0 0 1 +0 0 2 +0 0 2 +1 0 0 +1 0 0 +1 0 1 +1 0 1 +1 0 2 +1 0 2 +2 0 0 +2 0 0 +2 0 1 +2 0 1 +2 0 2 +2 0 2 +select * from t1 where a = 0 order by id,a,b; +id a b +0 0 0 +0 0 0 +0 0 1 +0 0 1 +0 0 2 +0 0 2 +1 0 0 +1 0 0 +1 0 1 +1 0 1 +1 0 2 +1 0 2 +2 0 0 +2 0 0 +2 0 1 +2 0 1 +2 0 2 +2 0 2 +select * from t1 where b = 0 order by id,a,b; +id a b +0 0 0 +0 0 0 +0 1 0 +0 1 0 +0 2 0 +0 2 0 +1 0 0 +1 0 0 +1 1 0 +1 1 0 +1 2 0 +1 2 0 +2 0 0 +2 0 0 +2 1 0 +2 1 0 +2 2 0 +2 2 0 +select * from t1 where b = 0 order by id,a,b; +id a b +0 0 0 +0 0 0 +0 1 0 +0 1 0 +0 2 0 +0 2 0 +1 0 0 +1 0 0 +1 1 0 +1 1 0 +1 2 0 +1 2 0 +2 0 0 +2 0 0 +2 1 0 +2 1 0 +2 2 0 +2 2 0 +select * from t1 where id = 1 order by id,a,b; +id a b +1 0 0 +1 0 0 +1 0 1 +1 0 1 +1 0 2 +1 0 2 +1 1 0 +1 1 0 +1 1 1 +1 1 1 +1 1 2 +1 1 2 +1 2 0 +1 2 0 +1 2 1 +1 2 1 +1 2 2 +1 2 2 +select * from t1 where id = 1 order by id,a,b; +id a b +1 0 0 +1 0 0 +1 0 1 +1 0 1 +1 0 2 +1 0 2 +1 1 0 +1 1 0 +1 1 1 +1 1 1 +1 1 2 +1 1 2 +1 2 0 +1 2 0 +1 2 1 +1 2 1 +1 2 2 +1 2 2 +select * from t1 where a = 1 order by id,a,b; +id a b +0 1 0 +0 1 0 +0 1 1 +0 1 1 +0 1 2 +0 1 2 +1 1 0 +1 1 0 +1 1 1 +1 1 1 +1 1 2 +1 1 2 +2 1 0 +2 1 0 +2 1 1 +2 1 1 +2 1 2 +2 1 2 +select * from t1 where a = 1 order by id,a,b; +id a b +0 1 0 +0 1 0 +0 1 1 +0 1 1 +0 1 2 +0 1 2 +1 1 0 +1 1 0 +1 1 1 +1 1 1 +1 1 2 +1 1 2 +2 1 0 +2 1 0 +2 1 1 +2 1 1 +2 1 2 +2 1 2 +select * from t1 where b = 1 order by id,a,b; +id a b +0 0 1 +0 0 1 +0 1 1 +0 1 1 +0 2 1 +0 2 1 +1 0 1 +1 0 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +2 0 1 +2 0 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +select * from t1 where b = 1 order by id,a,b; +id a b +0 0 1 +0 0 1 +0 1 1 +0 1 1 +0 2 1 +0 2 1 +1 0 1 +1 0 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +2 0 1 +2 0 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +select * from t1 where id = 2 order by id,a,b; +id a b +2 0 0 +2 0 0 +2 0 1 +2 0 1 +2 0 2 +2 0 2 +2 1 0 +2 1 0 +2 1 1 +2 1 1 +2 1 2 +2 1 2 +2 2 0 +2 2 0 +2 2 1 +2 2 1 +2 2 2 +2 2 2 +select * from t1 where id = 2 order by id,a,b; +id a b +2 0 0 +2 0 0 +2 0 1 +2 0 1 +2 0 2 +2 0 2 +2 1 0 +2 1 0 +2 1 1 +2 1 1 +2 1 2 +2 1 2 +2 2 0 +2 2 0 +2 2 1 +2 2 1 +2 2 2 +2 2 2 +select * from t1 where a = 2 order by id,a,b; +id a b +0 2 0 +0 2 0 +0 2 1 +0 2 1 +0 2 2 +0 2 2 +1 2 0 +1 2 0 +1 2 1 +1 2 1 +1 2 2 +1 2 2 +2 2 0 +2 2 0 +2 2 1 +2 2 1 +2 2 2 +2 2 2 +select * from t1 where a = 2 order by id,a,b; +id a b +0 2 0 +0 2 0 +0 2 1 +0 2 1 +0 2 2 +0 2 2 +1 2 0 +1 2 0 +1 2 1 +1 2 1 +1 2 2 +1 2 2 +2 2 0 +2 2 0 +2 2 1 +2 2 1 +2 2 2 +2 2 2 +select * from t1 where b = 2 order by id,a,b; +id a b +0 0 2 +0 0 2 +0 1 2 +0 1 2 +0 2 2 +0 2 2 +1 0 2 +1 0 2 +1 1 2 +1 1 2 +1 2 2 +1 2 2 +2 0 2 +2 0 2 +2 1 2 +2 1 2 +2 2 2 +2 2 2 +select * from t1 where b = 2 order by id,a,b; +id a b +0 0 2 +0 0 2 +0 1 2 +0 1 2 +0 2 2 +0 2 2 +1 0 2 +1 0 2 +1 1 2 +1 1 2 +1 2 2 +1 2 2 +2 0 2 +2 0 2 +2 1 2 +2 1 2 +2 2 2 +2 2 2 +select * from t1 where id = 3 order by id,a,b; +id a b +select * from t1 where id = 3 order by id,a,b; +id a b +select * from t1 where a = 3 order by id,a,b; +id a b +select * from t1 where a = 3 order by id,a,b; +id a b +select * from t1 where b = 3 order by id,a,b; +id a b +select * from t1 where b = 3 order by id,a,b; +id a b +select * from t1 where id = 4 order by id,a,b; +id a b +select * from t1 where id = 4 order by id,a,b; +id a b +select * from t1 where a = 4 order by id,a,b; +id a b +select * from t1 where a = 4 order by id,a,b; +id a b +select * from t1 where b = 4 order by id,a,b; +id a b +select * from t1 where b = 4 order by id,a,b; +id a b +select * from t1 where id = 5 order by id,a,b; +id a b +select * from t1 where id = 5 order by id,a,b; +id a b +select * from t1 where a = 5 order by id,a,b; +id a b +select * from t1 where a = 5 order by id,a,b; +id a b +select * from t1 where b = 5 order by id,a,b; +id a b +select * from t1 where b = 5 order by id,a,b; +id a b +select * from t1 where id = 6 order by id,a,b; +id a b +select * from t1 where id = 6 order by id,a,b; +id a b +select * from t1 where a = 6 order by id,a,b; +id a b +select * from t1 where a = 6 order by id,a,b; +id a b +select * from t1 where b = 6 order by id,a,b; +id a b +select * from t1 where b = 6 order by id,a,b; +id a b +select * from t1 where id = 7 order by id,a,b; +id a b +select * from t1 where id = 7 order by id,a,b; +id a b +select * from t1 where a = 7 order by id,a,b; +id a b +select * from t1 where a = 7 order by id,a,b; +id a b +select * from t1 where b = 7 order by id,a,b; +id a b +select * from t1 where b = 7 order by id,a,b; +id a b +select * from t1 where id = 8 order by id,a,b; +id a b +select * from t1 where id = 8 order by id,a,b; +id a b +select * from t1 where a = 8 order by id,a,b; +id a b +select * from t1 where a = 8 order by id,a,b; +id a b +select * from t1 where b = 8 order by id,a,b; +id a b +select * from t1 where b = 8 order by id,a,b; +id a b +select * from t1 where id = 9 order by id,a,b; +id a b +select * from t1 where id = 9 order by id,a,b; +id a b +select * from t1 where a = 9 order by id,a,b; +id a b +select * from t1 where a = 9 order by id,a,b; +id a b +select * from t1 where b = 9 order by id,a,b; +id a b +select * from t1 where b = 9 order by id,a,b; +id a b +select * from t1 where id = 10 order by id,a,b; +id a b +select * from t1 where id = 10 order by id,a,b; +id a b +select * from t1 where a = 10 order by id,a,b; +id a b +select * from t1 where a = 10 order by id,a,b; +id a b +select * from t1 where b = 10 order by id,a,b; +id a b +select * from t1 where b = 10 order by id,a,b; +id a b +select * from t1 where 0 = a or 4 = b order by id,a,b; +id a b +0 0 0 +0 0 0 +0 0 1 +0 0 1 +0 0 2 +0 0 2 +1 0 0 +1 0 0 +1 0 1 +1 0 1 +1 0 2 +1 0 2 +2 0 0 +2 0 0 +2 0 1 +2 0 1 +2 0 2 +2 0 2 +select * from t1 where 0 = a or 4 = b order by id,a,b; +id a b +0 0 0 +0 0 0 +0 0 1 +0 0 1 +0 0 2 +0 0 2 +1 0 0 +1 0 0 +1 0 1 +1 0 1 +1 0 2 +1 0 2 +2 0 0 +2 0 0 +2 0 1 +2 0 1 +2 0 2 +2 0 2 +select * from t1 where b in (3,4,3,1) and b = 0 order by id,a,b; +id a b +select * from t1 where b in (3,4,3,1) and b = 0 order by id,a,b; +id a b +select * from t1 where 1 = b and id = 3 and 1 = id and b in (1,0,1,3,4,0,4,4) order by id,a,b; +id a b +select * from t1 where 1 = b and id = 3 and 1 = id and b in (1,0,1,3,4,0,4,4) order by id,a,b; +id a b +select * from t1 where 1 = b and id in (1,1,4,4,1,0,3) order by id,a,b; +id a b +0 0 1 +0 0 1 +0 1 1 +0 1 1 +0 2 1 +0 2 1 +1 0 1 +1 0 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +select * from t1 where 1 = b and id in (1,1,4,4,1,0,3) order by id,a,b; +id a b +0 0 1 +0 0 1 +0 1 1 +0 1 1 +0 2 1 +0 2 1 +1 0 1 +1 0 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +select * from t1 where 1 = b and b = 4 order by id,a,b; +id a b +select * from t1 where 1 = b and b = 4 order by id,a,b; +id a b +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 ( +a int(11) DEFAULT NULL, +b int(11) DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH( a ) +PARTITIONS 4; +SELECT (SELECT tt.a FROM t1 tt ORDER BY a ASC LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa; +aa COUNT(DISTINCT b) +insert into t1 values (1, 1); +SELECT (SELECT tt.a FROM t1 tt ORDER BY a ASC LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa; +aa COUNT(DISTINCT b) +1 1 +insert into t1 values (2, 2), (2, 2); +SELECT (SELECT tt.a FROM t1 tt ORDER BY a ASC LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa; +aa COUNT(DISTINCT b) +1 2 +insert into t1 values (3, 3), (3, 3), (3, 3); +SELECT (SELECT tt.a FROM t1 tt ORDER BY a ASC LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa; +aa COUNT(DISTINCT b) +1 3 +insert into t1 values (4, 4), (4, 4), (4, 4), (4, 4); +SELECT (SELECT tt.a FROM t1 tt ORDER BY a DESC LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa; +aa COUNT(DISTINCT b) +4 4 +DROP TABLE IF EXISTS test; +CREATE TABLE NT_RP3763 (COL1 TINYINT(8) SIGNED COMMENT "NUMERIC NO INDEX" DEFAULT 41,COL2 VARCHAR(20),COL3 DATETIME,COL4 BIGINT,COL5 FLOAT) PARTITION BY RANGE (COL1 * COL3) (PARTITION P0 VALUES LESS THAN (0),PARTITION P1 VALUES LESS THAN (10),PARTITION P2 VALUES LESS THAN (20),PARTITION P3 VALUES LESS THAN (30),PARTITION P4 VALUES LESS THAN (40),PARTITION P5 VALUES LESS THAN (50),PARTITION PMX VALUES LESS THAN MAXVALUE); +insert into NT_RP3763 (COL1,COL2,COL3,COL4,COL5) values(-82,"夐齏醕皆磹漋甓崘潮嵙燷渏艂朼洛炷鉢儝鱈肇","5748\-06\-26\ 20:48:49",-3133527360541070260,-2.624880003397658e+38); +insert into NT_RP3763 (COL1,COL2,COL3,COL4,COL5) values(48,"簖鹩筈匹眜赖泽騈爷詵赺玡婙Ɇ郝鮙廛賙疼舢","7228\-12\-13\ 02:59:54",-6181009269190017937,2.7731105531290494e+38); +select * from `NT_RP3763` where `COL1` in (10, 48, -82); +COL1 COL2 COL3 COL4 COL5 +-82 夐齏醕皆磹漋甓崘潮嵙燷渏艂朼洛炷鉢儝鱈肇 5748-06-26 20:48:49 -3133527360541070260 -2.62488e38 +48 簖鹩筈匹眜赖泽騈爷詵赺玡婙Ɇ郝鮙廛賙疼舢 7228-12-13 02:59:54 -6181009269190017937 2.77311e38 +select * from `NT_RP3763` where `COL1` in (48); +COL1 COL2 COL3 COL4 COL5 +48 簖鹩筈匹眜赖泽騈爷詵赺玡婙Ɇ郝鮙廛賙疼舢 7228-12-13 02:59:54 -6181009269190017937 2.77311e38 +drop table if exists t2; +create table t2 (a int, b int) partition by range (a) (partition p0 values less than (0), partition p1 values less than (5)); +insert into t2(a) values (-1), (1); +select * from t2 where a > 10 or b is NULL order by a; +a b +-1 NULL +1 NULL +DROP TABLE IF EXISTS test; +CREATE TABLE test(a INT, b INT, PRIMARY KEY(a, b)) PARTITION BY RANGE (a + b) (PARTITION p0 VALUES LESS THAN (20),PARTITION p1 VALUES LESS THAN MAXVALUE); +INSERT INTO test(a, b) VALUES(1, 11),(2, 22),(3, 33),(10, 44),(9, 55); +select * FROM test WHERE a = 1; +a b +1 11 +select * FROM test WHERE b = 1; +a b +select * FROM test WHERE a = 1 AND b = 1; +a b +select * FROM test WHERE a + b = 2; +a b +set @@tidb_partition_prune_mode='static'; +drop table if exists t1; +create table t1(a int) partition by hash (a) partitions 10; +insert into t1 values (1), (2), (12), (3), (11), (13); +select * from t1 where a not between 2 and 2; +a +1 +11 +12 +13 +3 +select * from t1 where not (a < -20 or a > 20); +a +1 +11 +12 +13 +2 +3 +select * from t1 where not (a > 0 and a < 10); +a +11 +12 +13 +select * from t1 where not (a < -20); +a +1 +11 +12 +13 +2 +3 +select * from t1 where not (a > 20); +a +1 +11 +12 +13 +2 +3 +select * from t1 where not (a = 1); +a +11 +12 +13 +2 +3 +select * from t1 where not (a != 1); +a +1 +drop table if exists t2; +create table t2(a int) +partition by range (a) ( +partition p0 values less than (0), +partition p1 values less than (10), +partition p2 values less than (20) +); +explain format = 'brief' select * from t2 where not (a < 5); +id estRows task access object operator info +PartitionUnion 6666.67 root +├─TableReader 3333.33 root data:Selection +│ └─Selection 3333.33 cop[tikv] ge(test_partition_1.t2.a, 5) +│ └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p1 keep order:false, stats:pseudo +└─TableReader 3333.33 root data:Selection + └─Selection 3333.33 cop[tikv] ge(test_partition_1.t2.a, 5) + └─TableFullScan 10000.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo +set @@tidb_partition_prune_mode='dynamic'; +drop table if exists t3; +create table t3(a int) partition by hash (a) partitions 10; +insert into t3 values (1), (2), (12), (3), (11), (13); +select * from t3 where a not between 2 and 2; +a +1 +11 +12 +13 +3 +select * from t3 where not (a < -20 or a > 20); +a +1 +11 +12 +13 +2 +3 +select * from t3 where not (a > 0 and a < 10); +a +11 +12 +13 +select * from t3 where not (a < -20); +a +1 +11 +12 +13 +2 +3 +select * from t3 where not (a > 20); +a +1 +11 +12 +13 +2 +3 +select * from t3 where not (a = 1); +a +11 +12 +13 +2 +3 +select * from t3 where not (a != 1); +a +1 +set @@tidb_partition_prune_mode='static'; +DROP TABLE IF EXISTS t; +CREATE TABLE t (`COL1` int, `COL3` bigint) PARTITION BY HASH ((`COL1` * `COL3`))PARTITIONS 13; +select * FROM t WHERE col3 =2659937067964964513 and col1 = 783367513002; +COL1 COL3 +drop table if exists t; +CREATE TABLE `t` (`COL1` int NOT NULL DEFAULT '25' COMMENT 'NUMERIC PK',`COL3` bigint NOT NULL,PRIMARY KEY (`COL1`,`COL3`)) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin PARTITION BY HASH ((`COL1` * `COL3`))PARTITIONS 13; +insert into t(col1, col3) values(0, 3522101843073676459); +SELECT col1, COL3 FROM t WHERE COL1 IN (0,14158354938390,0) AND COL3 IN (3522101843073676459,-2846203247576845955,838395691793635638); +col1 COL3 +0 3522101843073676459 +set @@tidb_partition_prune_mode='dynamic'; +DROP TABLE IF EXISTS t; +create table t (a int primary key, b int, key (b)) partition by hash(a) (partition P0, partition p1, partition P2); +insert into t values (1, 1),(2, 2),(3, 3); +explain select * from t where a IN (1, 2); +id estRows task access object operator info +Batch_Point_Get_1 2.00 root table:t, partition:p1,P2 handle:[1 2], keep order:false, desc:false +explain select * from t where a IN (1, 2, 1); +id estRows task access object operator info +Batch_Point_Get_1 3.00 root table:t, partition:p1,P2 handle:[1 2], keep order:false, desc:false +create database Issue32007; +USE Issue32007; +create table t1 (a int, b tinyint, primary key (a)) partition by range (a) (partition p0 values less than (5),partition p1 values less than (20),partition p2 values less than (30),partition p3 values less than (40),partition p4 values less than MAXVALUE); +insert into t1 values (0, 0), (1, 1), (2, 2), (3, 3), (4, 4), (5, 5), (6, 6), (7, 7), (10, 10), (11, 11), (12, 12), (13, 13), (14, 14), (15, 15), (20, 20), (21, 21), (22, 22), (23, 23), (24, 24), (25, 25), (30, 30), (31, 31), (32, 32), (33, 33), (34, 34), (35, 35), (36, 36), (40, 40), (50, 50), (80, 80), (90, 90), (100, 100); +create table t3 (a int, b mediumint, primary key (a)); +insert into t3 values (0, 0), (1, 1), (2, 2), (3, 3), (4, 4), (5, 5), (6, 6), (7, 7), (8, 8), (9, 9), (10, 10), (11, 11), (12, 12), (13, 13), (14, 14), (15, 15), (16, 16), (17, 17), (18, 18), (19, 19), (20, 20), (21, 21), (22, 22), (23, 23); +set @@tidb_partition_prune_mode='static'; +select * from t3 where t3.a <> ALL (select t1.a from t1 partition (p0)) order by t3.a; +a b +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 +21 21 +22 22 +23 23 +5 5 +6 6 +7 7 +8 8 +9 9 +set @@tidb_partition_prune_mode='dynamic'; +select * from t3 where t3.a <> ALL (select t1.a from t1 partition (p0)) order by t3.a; +a b +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 +21 21 +22 22 +23 23 +5 5 +6 6 +7 7 +8 8 +9 9 +create database issue33231; +use issue33231; +set @@session.tidb_partition_prune_mode = 'dynamic'; +create table t1 (c_int int, c_str varchar(40), primary key (c_int, c_str) clustered, key(c_int) ) partition by hash (c_int) partitions 4; +create table t2 like t1; +insert into t1 values(6, 'beautiful curran'); +insert into t1 values(7, 'epic kalam'); +insert into t1 values(7, 'affectionate curie'); +insert into t2 values(6, 'vigorous rhodes'); +insert into t2 values(7, 'sweet aryabhata'); +select /*+ INL_JOIN(t2) */ * from t1, t2 where t1.c_int = t2.c_int and t1.c_str <= t2.c_str and t2.c_int in (6, 7, 6); +c_int c_str c_int c_str +6 beautiful curran 6 vigorous rhodes +7 affectionate curie 7 sweet aryabhata +7 epic kalam 7 sweet aryabhata +create database ListDefaultPrune; +use ListDefaultPrune; +create table t (a int, b int) partition by list columns (a,b) (partition p1 values in ((1,1)), partition p2 values in ((2,2)), partition pDef default); +insert into t values (1,1),(2,2),(1,2),(2,1),(3,3),(2,3),(1,4); +analyze table t; +select * from t where a in (1,2) and b in (1,2); +a b +1 1 +1 2 +2 1 +2 2 +select * from t where a in (1,2) and b in (3,4); +a b +1 4 +2 3 +explain format='brief' select * from t where a in (1,2) and b in (3,4); +id estRows task access object operator info +TableReader 2.57 root partition:pDef data:Selection +└─Selection 2.57 cop[tikv] in(listdefaultprune.t.a, 1, 2), in(listdefaultprune.t.b, 3, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +select * from t where a in (1,2) and b in (1,2); +a b +1 1 +1 2 +2 1 +2 2 +explain format='brief' select * from t where a in (1,2) and b in (1,2); +id estRows task access object operator info +TableReader 3.43 root partition:p1,p2,pDef data:Selection +└─Selection 3.43 cop[tikv] in(listdefaultprune.t.a, 1, 2), in(listdefaultprune.t.b, 1, 2) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +select * from t where a in (1) and b in (1); +a b +1 1 +explain format='brief' select * from t where a in (1) and b in (1); +id estRows task access object operator info +TableReader 0.86 root partition:p1,pDef data:Selection +└─Selection 0.86 cop[tikv] eq(listdefaultprune.t.a, 1), eq(listdefaultprune.t.b, 1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +select * from t where a = 1 and b = 1; +a b +1 1 +explain format='brief' select * from t where a = 1 and b = 1; +id estRows task access object operator info +TableReader 0.86 root partition:p1,pDef data:Selection +└─Selection 0.86 cop[tikv] eq(listdefaultprune.t.a, 1), eq(listdefaultprune.t.b, 1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +drop table t; +create table t (a int, b int) partition by list columns (a,b) (partition p1 values in ((1,1), (1,2)), partition p2 values in ((2,2),(2,1)), partition pDef default); +insert into t values (1,1),(2,2),(1,2),(2,1),(3,3),(2,3),(1,4); +analyze table t; +select * from t where a in (1,2) and b in (1,2); +a b +1 1 +1 2 +2 1 +2 2 +explain format='brief' select * from t where a in (1,2) and b in (1,2); +id estRows task access object operator info +TableReader 3.43 root partition:p1,p2,pDef data:Selection +└─Selection 3.43 cop[tikv] in(listdefaultprune.t.a, 1, 2), in(listdefaultprune.t.b, 1, 2) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +drop table t; +create table t (a int, b int) partition by list columns (a) (partition p1 values in (1), partition p2 values in (2), partition pDef default); +insert into t values (1,1),(2,2),(1,2),(2,1),(3,3),(2,3),(1,4); +analyze table t; +select * from t where a in (1,2); +a b +1 1 +1 2 +1 4 +2 1 +2 2 +2 3 +explain format='brief' select * from t where a in (1,2); +id estRows task access object operator info +TableReader 6.00 root partition:p1,p2 data:Selection +└─Selection 6.00 cop[tikv] in(listdefaultprune.t.a, 1, 2) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +select * from t where a = 1; +a b +1 1 +1 2 +1 4 +explain format='brief' select * from t where a = 1; +id estRows task access object operator info +TableReader 3.00 root partition:p1 data:Selection +└─Selection 3.00 cop[tikv] eq(listdefaultprune.t.a, 1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +create database issue42273; +use issue42273; +CREATE TABLE t(a tinyint unsigned, b tinyint unsigned) PARTITION BY RANGE COLUMNS (a,b)( +PARTITION p0 VALUES LESS THAN (10,255), +PARTITION p1 VALUES LESS THAN (20,MAXVALUE), +PARTITION p2 VALUES LESS THAN (30,255), +PARTITION p3 VALUES LESS THAN (MAXVALUE, 0)); +insert into t values(20, 30); +analyze table t; +explain format='brief' select * from t where a = 20; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] eq(issue42273.t.a, 20) + └─TableFullScan 1.00 cop[tikv] table:t keep order:false +explain format='brief' select * from t where a > 10 and a <= 20; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] gt(issue42273.t.a, 10), le(issue42273.t.a, 20) + └─TableFullScan 1.00 cop[tikv] table:t keep order:false +select * from t where a = 20; +a b +20 30 +select * from t where a > 10 and a <= 20; +a b +20 30 +drop database issue42273; +create database issue43459; +use issue43459; +set @@session.tidb_partition_prune_mode = 'dynamic'; +CREATE TABLE test1 (ID varchar(50) NOT NULL, +PARTITION_NO int(11) NOT NULL DEFAULT '0', +CREATE_TIME datetime NOT NULL DEFAULT CURRENT_TIMESTAMP, +PRIMARY KEY (ID,PARTITION_NO,CREATE_TIME), +KEY index_partition_no (PARTITION_NO) +) PARTITION BY RANGE COLUMNS(PARTITION_NO,CREATE_TIME) +(PARTITION 2023p1 VALUES LESS THAN (200000,'2023-01-01 00:00:00'), +PARTITION 2023p2 VALUES LESS THAN (300000,'2023-01-01 00:00:00')) ; +insert into test1 values("1", 200000, "2022-12-29 12:00:00"), ("2",200000,"2023-01-01"); +analyze table test1; +explain select * from test1 where partition_no > 199999; +id estRows task access object operator info +IndexReader_10 2.00 root partition:all index:Selection_9 +└─Selection_9 2.00 cop[tikv] gt(issue43459.test1.partition_no, 199999) + └─IndexFullScan_8 2.00 cop[tikv] table:test1, index:PRIMARY(ID, PARTITION_NO, CREATE_TIME) keep order:false +explain select * from test1 where partition_no = 200000; +id estRows task access object operator info +IndexReader_10 2.00 root partition:all index:Selection_9 +└─Selection_9 2.00 cop[tikv] eq(issue43459.test1.partition_no, 200000) + └─IndexFullScan_8 2.00 cop[tikv] table:test1, index:PRIMARY(ID, PARTITION_NO, CREATE_TIME) keep order:false +explain select * from test1 where partition_no >= 200000; +id estRows task access object operator info +IndexReader_10 2.00 root partition:all index:Selection_9 +└─Selection_9 2.00 cop[tikv] ge(issue43459.test1.partition_no, 200000) + └─IndexFullScan_8 2.00 cop[tikv] table:test1, index:PRIMARY(ID, PARTITION_NO, CREATE_TIME) keep order:false +explain select * from test1 where partition_no < 200000; +id estRows task access object operator info +IndexReader_10 0.00 root partition:2023p1 index:Selection_9 +└─Selection_9 0.00 cop[tikv] lt(issue43459.test1.partition_no, 200000) + └─IndexFullScan_8 2.00 cop[tikv] table:test1, index:PRIMARY(ID, PARTITION_NO, CREATE_TIME) keep order:false +explain select * from test1 where partition_no <= 200000; +id estRows task access object operator info +IndexReader_10 2.00 root partition:all index:Selection_9 +└─Selection_9 2.00 cop[tikv] le(issue43459.test1.partition_no, 200000) + └─IndexFullScan_8 2.00 cop[tikv] table:test1, index:PRIMARY(ID, PARTITION_NO, CREATE_TIME) keep order:false +explain select * from test1 where partition_no > 200000; +id estRows task access object operator info +IndexReader_10 0.00 root partition:2023p2 index:Selection_9 +└─Selection_9 0.00 cop[tikv] gt(issue43459.test1.partition_no, 200000) + └─IndexFullScan_8 2.00 cop[tikv] table:test1, index:PRIMARY(ID, PARTITION_NO, CREATE_TIME) keep order:false +select * from test1 partition (2023p1); +ID PARTITION_NO CREATE_TIME +1 200000 2022-12-29 12:00:00 +select * from test1 partition (2023p2); +ID PARTITION_NO CREATE_TIME +2 200000 2023-01-01 00:00:00 +select * from test1; +ID PARTITION_NO CREATE_TIME +1 200000 2022-12-29 12:00:00 +2 200000 2023-01-01 00:00:00 +select * from test1 where partition_no = 200000; +ID PARTITION_NO CREATE_TIME +1 200000 2022-12-29 12:00:00 +2 200000 2023-01-01 00:00:00 +select * from test1 where partition_no >= 200000; +ID PARTITION_NO CREATE_TIME +1 200000 2022-12-29 12:00:00 +2 200000 2023-01-01 00:00:00 +drop table test1; +CREATE TABLE test1 (ID varchar(50) NOT NULL, +PARTITION_NO int(11) NOT NULL DEFAULT '0', +CREATE_TIME date NOT NULL DEFAULT CURRENT_DATE, +PRIMARY KEY (ID,PARTITION_NO,CREATE_TIME), +KEY index_partition_no (PARTITION_NO) +) PARTITION BY RANGE COLUMNS(PARTITION_NO,CREATE_TIME) +(PARTITION 2023p1 VALUES LESS THAN (200000,'2023-01-01 00:00:00'), +PARTITION 2023p2 VALUES LESS THAN (300000,'2023-01-01 00:00:00')) ; +insert into test1 values("1", 200000, "2022-12-29 12:00:00"), ("2",200000,"2023-01-01"); +analyze table test1; +explain select * from test1 where partition_no > 199999; +id estRows task access object operator info +IndexReader_10 2.00 root partition:all index:Selection_9 +└─Selection_9 2.00 cop[tikv] gt(issue43459.test1.partition_no, 199999) + └─IndexFullScan_8 2.00 cop[tikv] table:test1, index:PRIMARY(ID, PARTITION_NO, CREATE_TIME) keep order:false +explain select * from test1 where partition_no = 200000; +id estRows task access object operator info +IndexReader_10 2.00 root partition:all index:Selection_9 +└─Selection_9 2.00 cop[tikv] eq(issue43459.test1.partition_no, 200000) + └─IndexFullScan_8 2.00 cop[tikv] table:test1, index:PRIMARY(ID, PARTITION_NO, CREATE_TIME) keep order:false +explain select * from test1 where partition_no >= 200000; +id estRows task access object operator info +IndexReader_10 2.00 root partition:all index:Selection_9 +└─Selection_9 2.00 cop[tikv] ge(issue43459.test1.partition_no, 200000) + └─IndexFullScan_8 2.00 cop[tikv] table:test1, index:PRIMARY(ID, PARTITION_NO, CREATE_TIME) keep order:false +explain select * from test1 where partition_no < 200000; +id estRows task access object operator info +IndexReader_10 0.00 root partition:2023p1 index:Selection_9 +└─Selection_9 0.00 cop[tikv] lt(issue43459.test1.partition_no, 200000) + └─IndexFullScan_8 2.00 cop[tikv] table:test1, index:PRIMARY(ID, PARTITION_NO, CREATE_TIME) keep order:false +explain select * from test1 where partition_no <= 200000; +id estRows task access object operator info +IndexReader_10 2.00 root partition:all index:Selection_9 +└─Selection_9 2.00 cop[tikv] le(issue43459.test1.partition_no, 200000) + └─IndexFullScan_8 2.00 cop[tikv] table:test1, index:PRIMARY(ID, PARTITION_NO, CREATE_TIME) keep order:false +explain select * from test1 where partition_no > 200000; +id estRows task access object operator info +IndexReader_10 0.00 root partition:2023p2 index:Selection_9 +└─Selection_9 0.00 cop[tikv] gt(issue43459.test1.partition_no, 200000) + └─IndexFullScan_8 2.00 cop[tikv] table:test1, index:PRIMARY(ID, PARTITION_NO, CREATE_TIME) keep order:false +select * from test1 partition (2023p1); +ID PARTITION_NO CREATE_TIME +1 200000 2022-12-29 +select * from test1 partition (2023p2); +ID PARTITION_NO CREATE_TIME +2 200000 2023-01-01 +select * from test1; +ID PARTITION_NO CREATE_TIME +1 200000 2022-12-29 +2 200000 2023-01-01 +select * from test1 where partition_no = 200000; +ID PARTITION_NO CREATE_TIME +1 200000 2022-12-29 +2 200000 2023-01-01 +select * from test1 where partition_no >= 200000; +ID PARTITION_NO CREATE_TIME +1 200000 2022-12-29 +2 200000 2023-01-01 +drop database issue43459; +use planner__core__partition_pruner; +drop table if exists t; +create table t(a bigint unsigned) +PARTITION BY RANGE (`a`) +(PARTITION `p0` VALUES LESS THAN (5086706), +PARTITION `p1` VALUES LESS THAN (7268292), +PARTITION `p2` VALUES LESS THAN (16545422), +PARTITION `p3` VALUES LESS THAN (9223372036854775810)); +desc select * from t where a BETWEEN -6895222 AND 3125507; +id estRows task access object operator info +TableReader_7 250.00 root partition:p0 data:Selection_6 +└─Selection_6 250.00 cop[tikv] ge(planner__core__partition_pruner.t.a, -6895222), le(planner__core__partition_pruner.t.a, 3125507) + └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +desc select * from t where a > 9223372036854775808; +id estRows task access object operator info +TableReader_7 3333.33 root partition:p3 data:Selection_6 +└─Selection_6 3333.33 cop[tikv] gt(planner__core__partition_pruner.t.a, 9223372036854775808) + └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +desc select * from t where a in (-6895222, 3125507, 9223372036854775809); +id estRows task access object operator info +TableReader_7 20.00 root partition:p0,p3 data:Selection_6 +└─Selection_6 20.00 cop[tikv] in(planner__core__partition_pruner.t.a, -6895222, 3125507, 9223372036854775809) + └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +drop table if exists t; +create table t(a bigint) +PARTITION BY RANGE (`a`) +(PARTITION `p0` VALUES LESS THAN (5086706), +PARTITION `p1` VALUES LESS THAN (7268292), +PARTITION `p2` VALUES LESS THAN (16545422), +PARTITION `p3` VALUES LESS THAN (9223372036854775807)); +desc select * from t where a BETWEEN -6895222 AND 3125507; +id estRows task access object operator info +TableReader_7 250.00 root partition:p0 data:Selection_6 +└─Selection_6 250.00 cop[tikv] ge(planner__core__partition_pruner.t.a, -6895222), le(planner__core__partition_pruner.t.a, 3125507) + └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +desc select * from t where a > 9223372036854775808; +id estRows task access object operator info +TableReader_7 0.00 root partition:dual data:Selection_6 +└─Selection_6 0.00 cop[tikv] gt(planner__core__partition_pruner.t.a, 9223372036854775808) + └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +desc select * from t where a in (-6895222, 3125507, 9223372036854775809); +id estRows task access object operator info +TableReader_7 20.00 root partition:p0 data:Selection_6 +└─Selection_6 20.00 cop[tikv] in(planner__core__partition_pruner.t.a, -6895222, 3125507, 9223372036854775809) + └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index ad83f68ebd258..13e8a1b55d0f1 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -169,11 +169,20 @@ func convertPoint(sctx sessionctx.Context, point *point, tp *types.FieldType) (* // see issue #20101: overflow when converting integer to year } else if tp.GetType() == mysql.TypeBit && terror.ErrorEqual(err, types.ErrDataTooLong) { // see issue #19067: we should ignore the types.ErrDataTooLong when we convert value to TypeBit value +<<<<<<< HEAD:util/ranger/ranger.go } else if tp.GetType() == mysql.TypeNewDecimal && terror.ErrorEqual(err, types.ErrOverflow) { // Ignore the types.ErrOverflow when we convert TypeNewDecimal values. // A trimmed valid boundary point value would be returned then. Accordingly, the `excl` of the point // would be adjusted. Impossible ranges would be skipped by the `validInterval` call later. } else if point.value.Kind() == types.KindMysqlTime && tp.GetType() == mysql.TypeTimestamp && terror.ErrorEqual(err, types.ErrWrongValue) { +======= + } else if (newTp.GetType() == mysql.TypeNewDecimal || newTp.GetType() == mysql.TypeLonglong) && terror.ErrorEqual(err, types.ErrOverflow) { + // Ignore the types.ErrOverflow when we convert TypeNewDecimal/TypeLonglong values. + // A trimmed valid boundary point value would be returned then. Accordingly, the `excl` of the point + // would be adjusted. Impossible ranges would be skipped by the `validInterval` call later. + // tests in TestIndexRange/TestIndexRangeForDecimal + } else if point.value.Kind() == types.KindMysqlTime && newTp.GetType() == mysql.TypeTimestamp && terror.ErrorEqual(err, types.ErrWrongValue) { +>>>>>>> fe5858b00cd (ranger: handle longlong overflow properly (#52365)):pkg/util/ranger/ranger.go // See issue #28424: query failed after add index // Ignore conversion from Date[Time] to Timestamp since it must be either out of range or impossible date, which will not match a point select } else if tp.GetType() == mysql.TypeEnum && terror.ErrorEqual(err, types.ErrTruncated) {