diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 3cf445e547a14..7601a4d4c41ae 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -744,8 +744,13 @@ insert into t values (1),(2),(2),(2),(9),(9),(9),(10); analyze table t with 1 buckets; explain format = 'brief' select * from t where a >= 3 and a <= 8; id estRows task access object operator info +<<<<<<< HEAD:cmd/explaintest/r/explain_easy.result TableReader 0.00 root data:Selection └─Selection 0.00 cop[tikv] ge(test.t.a, 3), le(test.t.a, 8) +======= +TableReader 1.00 root data:Selection +└─Selection 1.00 cop[tikv] ge(explain_easy.t.a, 3), le(explain_easy.t.a, 8) +>>>>>>> f2c278ddc6b (Planner: Do not allow cardinality to go below 1 (#55242)):tests/integrationtest/r/explain_easy.result └─TableFullScan 8.00 cop[tikv] table:t keep order:false drop table t; create table t(a int, b int, index idx_ab(a, b)); diff --git a/cmd/explaintest/r/explain_generate_column_substitute.result b/cmd/explaintest/r/explain_generate_column_substitute.result index bdbc22b92ade1..ce5ef5dfcc816 100644 --- a/cmd/explaintest/r/explain_generate_column_substitute.result +++ b/cmd/explaintest/r/explain_generate_column_substitute.result @@ -440,10 +440,17 @@ Projection 1.00 root test.t.a, test.t.b └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false desc format = 'brief' select * from t where not (lower(b) >= "a"); id estRows task access object operator info +<<<<<<< HEAD:cmd/explaintest/r/explain_generate_column_substitute.result Projection 0.00 root test.t.a, test.t.b └─IndexLookUp 0.00 root ├─IndexRangeScan(Build) 0.00 cop[tikv] table:t, index:expression_index(lower(`b`), `a` + 1) range:[-inf,"a"), keep order:false └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false +======= +Projection 1.00 root explain_generate_column_substitute.t.a, explain_generate_column_substitute.t.b +└─IndexLookUp 1.00 root + ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:expression_index(lower(`b`), `a` + 1) range:[-inf,"a"), keep order:false + └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false +>>>>>>> f2c278ddc6b (Planner: Do not allow cardinality to go below 1 (#55242)):tests/integrationtest/r/explain_generate_column_substitute.result desc format = 'brief' select count(upper(b)) from t group by upper(b); id estRows task access object operator info StreamAgg 4.80 root group by:upper(test.t.b), funcs:count(upper(test.t.b))->Column#7 diff --git a/cmd/explaintest/r/imdbload.result b/cmd/explaintest/r/imdbload.result index 066dc14b155ae..073396cea759e 100644 --- a/cmd/explaintest/r/imdbload.result +++ b/cmd/explaintest/r/imdbload.result @@ -287,48 +287,48 @@ IndexLookUp_7 1005030.94 root └─TableRowIDScan_6(Probe) 1005030.94 cop[tikv] table:char_name keep order:false trace plan target = 'estimation' select * from char_name where ((imdb_index = 'I') and (surname_pcode < 'E436')) or ((imdb_index = 'L') and (surname_pcode < 'E436')); CE_trace -[{"table_name":"char_name","type":"Column Stats-Point","expr":"((imdb_index = 'I'))","row_count":0},{"table_name":"char_name","type":"Column Stats-Point","expr":"((imdb_index = 'L'))","row_count":0},{"table_name":"char_name","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":4314864},{"table_name":"char_name","type":"Column Stats-Range","expr":"((surname_pcode < 'E436'))","row_count":1005030},{"table_name":"char_name","type":"Index Stats-Range","expr":"((imdb_index = 'I') and (surname_pcode < 'E436')) or ((imdb_index = 'L') and (surname_pcode < 'E436'))","row_count":0},{"table_name":"char_name","type":"Index Stats-Range","expr":"((surname_pcode < 'E436'))","row_count":1005030},{"table_name":"char_name","type":"Table Stats-Expression-CNF","expr":"`or`(`and`(`eq`(imdbload.char_name.imdb_index, 'I'), `lt`(imdbload.char_name.surname_pcode, 'E436')), `and`(`eq`(imdbload.char_name.imdb_index, 'L'), `lt`(imdbload.char_name.surname_pcode, 'E436')))","row_count":804024}] +[{"table_name":"char_name","type":"Column Stats-Point","expr":"((imdb_index = 'I'))","row_count":1},{"table_name":"char_name","type":"Column Stats-Point","expr":"((imdb_index = 'L'))","row_count":1},{"table_name":"char_name","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":4314864},{"table_name":"char_name","type":"Column Stats-Range","expr":"((surname_pcode < 'E436'))","row_count":1005030},{"table_name":"char_name","type":"Index Stats-Range","expr":"((imdb_index = 'I') and (surname_pcode < 'E436')) or ((imdb_index = 'L') and (surname_pcode < 'E436'))","row_count":2},{"table_name":"char_name","type":"Index Stats-Range","expr":"((surname_pcode < 'E436'))","row_count":1005030},{"table_name":"char_name","type":"Table Stats-Expression-CNF","expr":"`or`(`and`(`eq`(imdbload.char_name.imdb_index, 'I'), `lt`(imdbload.char_name.surname_pcode, 'E436')), `and`(`eq`(imdbload.char_name.imdb_index, 'L'), `lt`(imdbload.char_name.surname_pcode, 'E436')))","row_count":804024}] explain select * from char_name where ((imdb_index = 'V') and (surname_pcode < 'L3416')); id estRows task access object operator info -IndexLookUp_10 0.00 root -├─IndexRangeScan_8(Build) 0.00 cop[tikv] table:char_name, index:itest2(imdb_index, surname_pcode, name_pcode_nf) range:["V" -inf,"V" "L3416"), keep order:false -└─TableRowIDScan_9(Probe) 0.00 cop[tikv] table:char_name keep order:false +IndexLookUp_10 1.00 root +├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:char_name, index:itest2(imdb_index, surname_pcode, name_pcode_nf) range:["V" -inf,"V" "L3416"), keep order:false +└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:char_name keep order:false explain select * from char_name where imdb_index > 'V'; id estRows task access object operator info -IndexLookUp_10 0.00 root -├─IndexRangeScan_8(Build) 0.00 cop[tikv] table:char_name, index:itest2(imdb_index, surname_pcode, name_pcode_nf) range:("V",+inf], keep order:false -└─TableRowIDScan_9(Probe) 0.00 cop[tikv] table:char_name keep order:false +IndexLookUp_10 1.00 root +├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:char_name, index:itest2(imdb_index, surname_pcode, name_pcode_nf) range:("V",+inf], keep order:false +└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:char_name keep order:false trace plan target = 'estimation' select * from char_name where imdb_index > 'V'; CE_trace -[{"table_name":"char_name","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":4314864},{"table_name":"char_name","type":"Column Stats-Range","expr":"((imdb_index > 'V' and true))","row_count":0},{"table_name":"char_name","type":"Index Stats-Range","expr":"((imdb_index > 'V' and true))","row_count":0},{"table_name":"char_name","type":"Table Stats-Expression-CNF","expr":"`gt`(imdbload.char_name.imdb_index, 'V')","row_count":0}] +[{"table_name":"char_name","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":4314864},{"table_name":"char_name","type":"Column Stats-Range","expr":"((imdb_index > 'V' and true))","row_count":1},{"table_name":"char_name","type":"Index Stats-Range","expr":"((imdb_index > 'V' and true))","row_count":1},{"table_name":"char_name","type":"Table Stats-Expression-CNF","expr":"`gt`(imdbload.char_name.imdb_index, 'V')","row_count":1}] explain select * from movie_companies where company_type_id > 2; id estRows task access object operator info -IndexLookUp_10 0.00 root -├─IndexRangeScan_8(Build) 0.00 cop[tikv] table:movie_companies, index:movie_companies_idx_ctypeid(company_type_id) range:(2,+inf], keep order:false -└─TableRowIDScan_9(Probe) 0.00 cop[tikv] table:movie_companies keep order:false +IndexLookUp_10 1.00 root +├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:movie_companies, index:movie_companies_idx_ctypeid(company_type_id) range:(2,+inf], keep order:false +└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:movie_companies keep order:false trace plan target = 'estimation' select * from movie_companies where company_type_id > 2; CE_trace -[{"table_name":"movie_companies","type":"Column Stats-Range","expr":"((company_type_id > 2 and true))","row_count":0},{"table_name":"movie_companies","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":4958296},{"table_name":"movie_companies","type":"Index Stats-Range","expr":"((company_type_id > 2 and true))","row_count":0},{"table_name":"movie_companies","type":"Table Stats-Expression-CNF","expr":"`gt`(imdbload.movie_companies.company_type_id, 2)","row_count":0}] +[{"table_name":"movie_companies","type":"Column Stats-Range","expr":"((company_type_id > 2 and true))","row_count":1},{"table_name":"movie_companies","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":4958296},{"table_name":"movie_companies","type":"Index Stats-Range","expr":"((company_type_id > 2 and true))","row_count":1},{"table_name":"movie_companies","type":"Table Stats-Expression-CNF","expr":"`gt`(imdbload.movie_companies.company_type_id, 2)","row_count":1}] explain select * from char_name where imdb_index > 'I' and imdb_index < 'II'; id estRows task access object operator info -IndexLookUp_10 0.00 root -├─IndexRangeScan_8(Build) 0.00 cop[tikv] table:char_name, index:itest2(imdb_index, surname_pcode, name_pcode_nf) range:("I","II"), keep order:false -└─TableRowIDScan_9(Probe) 0.00 cop[tikv] table:char_name keep order:false +IndexLookUp_10 1.00 root +├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:char_name, index:itest2(imdb_index, surname_pcode, name_pcode_nf) range:("I","II"), keep order:false +└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:char_name keep order:false trace plan target = 'estimation' select * from char_name where imdb_index > 'I' and imdb_index < 'II'; CE_trace -[{"table_name":"char_name","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":4314864},{"table_name":"char_name","type":"Column Stats-Range","expr":"((imdb_index > 'I' and imdb_index < 'II'))","row_count":0},{"table_name":"char_name","type":"Index Stats-Range","expr":"((imdb_index > 'I' and imdb_index < 'II'))","row_count":0},{"table_name":"char_name","type":"Table Stats-Expression-CNF","expr":"`and`(`gt`(imdbload.char_name.imdb_index, 'I'), `lt`(imdbload.char_name.imdb_index, 'II'))","row_count":0}] +[{"table_name":"char_name","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":4314864},{"table_name":"char_name","type":"Column Stats-Range","expr":"((imdb_index > 'I' and imdb_index < 'II'))","row_count":1},{"table_name":"char_name","type":"Index Stats-Range","expr":"((imdb_index > 'I' and imdb_index < 'II'))","row_count":1},{"table_name":"char_name","type":"Table Stats-Expression-CNF","expr":"`and`(`gt`(imdbload.char_name.imdb_index, 'I'), `lt`(imdbload.char_name.imdb_index, 'II'))","row_count":1}] explain select * from char_name where imdb_index > 'I'; id estRows task access object operator info -IndexLookUp_10 0.00 root -├─IndexRangeScan_8(Build) 0.00 cop[tikv] table:char_name, index:itest2(imdb_index, surname_pcode, name_pcode_nf) range:("I",+inf], keep order:false -└─TableRowIDScan_9(Probe) 0.00 cop[tikv] table:char_name keep order:false +IndexLookUp_10 1.00 root +├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:char_name, index:itest2(imdb_index, surname_pcode, name_pcode_nf) range:("I",+inf], keep order:false +└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:char_name keep order:false trace plan target = 'estimation' select * from char_name where imdb_index > 'I'; CE_trace -[{"table_name":"char_name","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":4314864},{"table_name":"char_name","type":"Column Stats-Range","expr":"((imdb_index > 'I' and true))","row_count":0},{"table_name":"char_name","type":"Index Stats-Range","expr":"((imdb_index > 'I' and true))","row_count":0},{"table_name":"char_name","type":"Table Stats-Expression-CNF","expr":"`gt`(imdbload.char_name.imdb_index, 'I')","row_count":0}] +[{"table_name":"char_name","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":4314864},{"table_name":"char_name","type":"Column Stats-Range","expr":"((imdb_index > 'I' and true))","row_count":1},{"table_name":"char_name","type":"Index Stats-Range","expr":"((imdb_index > 'I' and true))","row_count":1},{"table_name":"char_name","type":"Table Stats-Expression-CNF","expr":"`gt`(imdbload.char_name.imdb_index, 'I')","row_count":1}] explain select * from cast_info where nr_order < -2068070866; id estRows task access object operator info @@ -342,12 +342,12 @@ IndexLookUp_10 34260.33 root └─TableRowIDScan_9(Probe) 34260.33 cop[tikv] table:aka_title keep order:false explain select * from aka_title where kind_id > 7; id estRows task access object operator info -IndexLookUp_10 0.00 root -├─IndexRangeScan_8(Build) 0.00 cop[tikv] table:aka_title, index:aka_title_idx_kindid(kind_id) range:(7,+inf], keep order:false -└─TableRowIDScan_9(Probe) 0.00 cop[tikv] table:aka_title keep order:false +IndexLookUp_10 1.00 root +├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:aka_title, index:aka_title_idx_kindid(kind_id) range:(7,+inf], keep order:false +└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:aka_title keep order:false trace plan target = 'estimation' select * from aka_title where kind_id > 7; CE_trace -[{"table_name":"aka_title","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":528337},{"table_name":"aka_title","type":"Column Stats-Range","expr":"((kind_id > 7 and true))","row_count":0},{"table_name":"aka_title","type":"Index Stats-Range","expr":"((kind_id > 7 and true))","row_count":0},{"table_name":"aka_title","type":"Table Stats-Expression-CNF","expr":"`gt`(imdbload.aka_title.kind_id, 7)","row_count":0}] +[{"table_name":"aka_title","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":528337},{"table_name":"aka_title","type":"Column Stats-Range","expr":"((kind_id > 7 and true))","row_count":1},{"table_name":"aka_title","type":"Index Stats-Range","expr":"((kind_id > 7 and true))","row_count":1},{"table_name":"aka_title","type":"Table Stats-Expression-CNF","expr":"`gt`(imdbload.aka_title.kind_id, 7)","row_count":1}] explain select * from keyword where ((phonetic_code = 'R1652') and (keyword > 'ecg-monitor' and keyword < 'killers')); id estRows task access object operator info diff --git a/cmd/explaintest/r/tpch.result b/cmd/explaintest/r/tpch.result index 28328e4c63241..8a72c35111c16 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -1298,6 +1298,7 @@ cntrycode; id estRows task access object operator info Sort 1.00 root Column#31 └─Projection 1.00 root Column#31, Column#32, Column#33 +<<<<<<< HEAD:cmd/explaintest/r/tpch.result └─HashAgg 1.00 root group by:Column#37, funcs:count(1)->Column#32, funcs:sum(Column#35)->Column#33, funcs:firstrow(Column#36)->Column#31 └─Projection 0.00 root tpch.customer.c_acctbal, substring(tpch.customer.c_phone, 1, 2)->Column#36, substring(tpch.customer.c_phone, 1, 2)->Column#37 └─HashJoin 0.00 root anti semi join, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] @@ -1305,4 +1306,13 @@ Sort 1.00 root Column#31 │ └─TableFullScan 75000000.00 cop[tikv] table:orders keep order:false └─TableReader(Probe) 0.00 root data:Selection └─Selection 0.00 cop[tikv] gt(tpch.customer.c_acctbal, NULL), in(substring(tpch.customer.c_phone, 1, 2), "20", "40", "22", "30", "39", "42", "21") +======= + └─HashAgg 1.00 root group by:Column#36, funcs:count(1)->Column#32, funcs:sum(Column#35)->Column#33, funcs:firstrow(Column#36)->Column#31 + └─Projection 0.64 root tpch50.customer.c_acctbal->Column#35, substring(tpch50.customer.c_phone, 1, 2)->Column#36 + └─HashJoin 0.64 root anti semi join, equal:[eq(tpch50.customer.c_custkey, tpch50.orders.o_custkey)] + ├─TableReader(Build) 75000000.00 root data:TableFullScan + │ └─TableFullScan 75000000.00 cop[tikv] table:orders keep order:false + └─TableReader(Probe) 0.80 root data:Selection + └─Selection 0.80 cop[tikv] gt(tpch50.customer.c_acctbal, NULL), in(substring(tpch50.customer.c_phone, 1, 2), "20", "40", "22", "30", "39", "42", "21") +>>>>>>> f2c278ddc6b (Planner: Do not allow cardinality to go below 1 (#55242)):tests/integrationtest/r/tpch.result └─TableFullScan 7500000.00 cop[tikv] table:customer keep order:false diff --git a/pkg/planner/cardinality/BUILD.bazel b/pkg/planner/cardinality/BUILD.bazel new file mode 100644 index 0000000000000..4446861b0e7a9 --- /dev/null +++ b/pkg/planner/cardinality/BUILD.bazel @@ -0,0 +1,95 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "cardinality", + srcs = [ + "cross_estimation.go", + "join.go", + "ndv.go", + "pseudo.go", + "row_count_column.go", + "row_count_index.go", + "row_size.go", + "selectivity.go", + "trace.go", + ], + importpath = "github.com/pingcap/tidb/pkg/planner/cardinality", + visibility = ["//visibility:public"], + deps = [ + "//pkg/expression", + "//pkg/kv", + "//pkg/parser/ast", + "//pkg/parser/format", + "//pkg/parser/model", + "//pkg/parser/mysql", + "//pkg/planner/context", + "//pkg/planner/property", + "//pkg/planner/util", + "//pkg/planner/util/debugtrace", + "//pkg/planner/util/fixcontrol", + "//pkg/sessionctx/stmtctx", + "//pkg/statistics", + "//pkg/tablecodec", + "//pkg/types", + "//pkg/types/parser_driver", + "//pkg/util/chunk", + "//pkg/util/codec", + "//pkg/util/collate", + "//pkg/util/logutil", + "//pkg/util/mathutil", + "//pkg/util/ranger", + "//pkg/util/set", + "//pkg/util/tracing", + "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_failpoint//:failpoint", + "@org_uber_go_zap//:zap", + ], +) + +go_test( + name = "cardinality_test", + timeout = "short", + srcs = [ + "main_test.go", + "row_count_test.go", + "row_size_test.go", + "selectivity_test.go", + "trace_test.go", + ], + data = glob(["testdata/**"]), + embed = [":cardinality"], + flaky = True, + shard_count = 28, + deps = [ + "//pkg/config", + "//pkg/domain", + "//pkg/executor", + "//pkg/expression", + "//pkg/infoschema", + "//pkg/kv", + "//pkg/parser", + "//pkg/parser/model", + "//pkg/parser/mysql", + "//pkg/planner/core", + "//pkg/planner/core/base", + "//pkg/planner/core/operator/logicalop", + "//pkg/session", + "//pkg/sessionctx", + "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/variable", + "//pkg/statistics", + "//pkg/testkit", + "//pkg/testkit/testdata", + "//pkg/testkit/testmain", + "//pkg/testkit/testsetup", + "//pkg/types", + "//pkg/util/codec", + "//pkg/util/collate", + "//pkg/util/mock", + "//pkg/util/ranger", + "//pkg/util/tracing", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/pkg/planner/cardinality/row_count_column.go b/pkg/planner/cardinality/row_count_column.go new file mode 100644 index 0000000000000..1f077e513cc54 --- /dev/null +++ b/pkg/planner/cardinality/row_count_column.go @@ -0,0 +1,397 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package cardinality + +import ( + "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/planner/context" + "github.com/pingcap/tidb/pkg/planner/util/debugtrace" + "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" + "github.com/pingcap/tidb/pkg/statistics" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/codec" + "github.com/pingcap/tidb/pkg/util/collate" + "github.com/pingcap/tidb/pkg/util/mathutil" + "github.com/pingcap/tidb/pkg/util/ranger" +) + +func init() { + statistics.GetRowCountByColumnRanges = GetRowCountByColumnRanges + statistics.GetRowCountByIntColumnRanges = GetRowCountByIntColumnRanges + statistics.GetRowCountByIndexRanges = GetRowCountByIndexRanges +} + +// GetRowCountByColumnRanges estimates the row count by a slice of Range. +func GetRowCountByColumnRanges(sctx context.PlanContext, coll *statistics.HistColl, colUniqueID int64, colRanges []*ranger.Range) (result float64, err error) { + var name string + if sctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(sctx) + debugTraceGetRowCountInput(sctx, colUniqueID, colRanges) + defer func() { + debugtrace.RecordAnyValuesWithNames(sctx, "Name", name, "Result", result) + debugtrace.LeaveContextCommon(sctx) + }() + } + sc := sctx.GetSessionVars().StmtCtx + c := coll.GetCol(colUniqueID) + colInfoID := colUniqueID + if len(coll.UniqueID2colInfoID) > 0 { + colInfoID = coll.UniqueID2colInfoID[colUniqueID] + } + recordUsedItemStatsStatus(sctx, c, coll.PhysicalID, colInfoID) + if c != nil && c.Info != nil { + name = c.Info.Name.O + } + if statistics.ColumnStatsIsInvalid(c, sctx, coll, colUniqueID) { + result, err = getPseudoRowCountByColumnRanges(sc.TypeCtx(), float64(coll.RealtimeCount), colRanges, 0) + if err == nil && sc.EnableOptimizerCETrace && c != nil { + ceTraceRange(sctx, coll.PhysicalID, []string{c.Info.Name.O}, colRanges, "Column Stats-Pseudo", uint64(result)) + } + return result, err + } + if sctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.RecordAnyValuesWithNames(sctx, + "Histogram NotNull Count", c.Histogram.NotNullCount(), + "TopN total count", c.TopN.TotalCount(), + "Increase Factor", c.GetIncreaseFactor(coll.RealtimeCount), + ) + } + result, err = GetColumnRowCount(sctx, c, colRanges, coll.RealtimeCount, coll.ModifyCount, false) + if sc.EnableOptimizerCETrace { + ceTraceRange(sctx, coll.PhysicalID, []string{c.Info.Name.O}, colRanges, "Column Stats", uint64(result)) + } + return result, errors.Trace(err) +} + +// GetRowCountByIntColumnRanges estimates the row count by a slice of IntColumnRange. +func GetRowCountByIntColumnRanges(sctx context.PlanContext, coll *statistics.HistColl, colUniqueID int64, intRanges []*ranger.Range) (result float64, err error) { + var name string + if sctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(sctx) + debugTraceGetRowCountInput(sctx, colUniqueID, intRanges) + defer func() { + debugtrace.RecordAnyValuesWithNames(sctx, "Name", name, "Result", result) + debugtrace.LeaveContextCommon(sctx) + }() + } + sc := sctx.GetSessionVars().StmtCtx + c := coll.GetCol(colUniqueID) + colInfoID := colUniqueID + if len(coll.UniqueID2colInfoID) > 0 { + colInfoID = coll.UniqueID2colInfoID[colUniqueID] + } + recordUsedItemStatsStatus(sctx, c, coll.PhysicalID, colInfoID) + if c != nil && c.Info != nil { + name = c.Info.Name.O + } + if statistics.ColumnStatsIsInvalid(c, sctx, coll, colUniqueID) { + if len(intRanges) == 0 { + return 0, nil + } + if intRanges[0].LowVal[0].Kind() == types.KindInt64 { + result = getPseudoRowCountBySignedIntRanges(intRanges, float64(coll.RealtimeCount)) + } else { + result = getPseudoRowCountByUnsignedIntRanges(intRanges, float64(coll.RealtimeCount)) + } + if sc.EnableOptimizerCETrace && c != nil { + ceTraceRange(sctx, coll.PhysicalID, []string{c.Info.Name.O}, intRanges, "Column Stats-Pseudo", uint64(result)) + } + return result, nil + } + if sctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.RecordAnyValuesWithNames(sctx, + "Histogram NotNull Count", c.Histogram.NotNullCount(), + "TopN total count", c.TopN.TotalCount(), + "Increase Factor", c.GetIncreaseFactor(coll.RealtimeCount), + ) + } + result, err = GetColumnRowCount(sctx, c, intRanges, coll.RealtimeCount, coll.ModifyCount, true) + if sc.EnableOptimizerCETrace { + ceTraceRange(sctx, coll.PhysicalID, []string{c.Info.Name.O}, intRanges, "Column Stats", uint64(result)) + } + return result, errors.Trace(err) +} + +// equalRowCountOnColumn estimates the row count by a slice of Range and a Datum. +func equalRowCountOnColumn(sctx context.PlanContext, c *statistics.Column, val types.Datum, encodedVal []byte, realtimeRowCount, modifyCount int64) (result float64, err error) { + if sctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(sctx) + debugtrace.RecordAnyValuesWithNames(sctx, "Value", val.String(), "Encoded", encodedVal) + defer func() { + debugtrace.RecordAnyValuesWithNames(sctx, "Result", result, "Error", err) + debugtrace.LeaveContextCommon(sctx) + }() + } + if val.IsNull() { + return float64(c.NullCount), nil + } + if c.StatsVer < statistics.Version2 { + // All the values are null. + if c.Histogram.Bounds.NumRows() == 0 { + return 0.0, nil + } + if c.Histogram.NDV > 0 && c.OutOfRange(val) { + return outOfRangeEQSelectivity(sctx, c.Histogram.NDV, realtimeRowCount, int64(c.TotalRowCount())) * c.TotalRowCount(), nil + } + if c.CMSketch != nil { + count, err := statistics.QueryValue(sctx, c.CMSketch, c.TopN, val) + return float64(count), errors.Trace(err) + } + histRowCount, _ := c.Histogram.EqualRowCount(sctx, val, false) + return histRowCount, nil + } + + // Stats version == 2 + // All the values are null. + if c.Histogram.Bounds.NumRows() == 0 && c.TopN.Num() == 0 { + return 0, nil + } + // 1. try to find this value in TopN + if c.TopN != nil { + rowcount, ok := c.TopN.QueryTopN(sctx, encodedVal) + if ok { + return float64(rowcount), nil + } + } + // 2. try to find this value in bucket.Repeat(the last value in every bucket) + histCnt, matched := c.Histogram.EqualRowCount(sctx, val, true) + if matched { + return histCnt, nil + } + // 3. use uniform distribution assumption for the rest (even when this value is not covered by the range of stats) + histNDV := float64(c.Histogram.NDV - int64(c.TopN.Num())) + if histNDV <= 0 { + // If the table hasn't been modified, it's safe to return 0. Otherwise, the TopN could be stale - return 1. + if modifyCount == 0 { + return 0, nil + } + return 1, nil + } + return c.Histogram.NotNullCount() / histNDV, nil +} + +// GetColumnRowCount estimates the row count by a slice of Range. +func GetColumnRowCount(sctx context.PlanContext, c *statistics.Column, ranges []*ranger.Range, realtimeRowCount, modifyCount int64, pkIsHandle bool) (float64, error) { + sc := sctx.GetSessionVars().StmtCtx + debugTrace := sc.EnableOptimizerDebugTrace + if debugTrace { + debugtrace.EnterContextCommon(sctx) + defer debugtrace.LeaveContextCommon(sctx) + } + var rowCount float64 + for _, rg := range ranges { + highVal := *rg.HighVal[0].Clone() + lowVal := *rg.LowVal[0].Clone() + if highVal.Kind() == types.KindString { + highVal.SetBytes(collate.GetCollator(highVal.Collation()).Key(highVal.GetString())) + } + if lowVal.Kind() == types.KindString { + lowVal.SetBytes(collate.GetCollator(lowVal.Collation()).Key(lowVal.GetString())) + } + cmp, err := lowVal.Compare(sc.TypeCtx(), &highVal, collate.GetBinaryCollator()) + if err != nil { + return 0, errors.Trace(err) + } + lowEncoded, err := codec.EncodeKey(sc.TimeZone(), nil, lowVal) + err = sc.HandleError(err) + if err != nil { + return 0, err + } + highEncoded, err := codec.EncodeKey(sc.TimeZone(), nil, highVal) + err = sc.HandleError(err) + if err != nil { + return 0, err + } + if debugTrace { + debugTraceStartEstimateRange(sctx, rg, lowEncoded, highEncoded, rowCount) + } + if cmp == 0 { + // case 1: it's a point + if !rg.LowExclude && !rg.HighExclude { + // In this case, the row count is at most 1. + if pkIsHandle { + rowCount++ + if debugTrace { + debugTraceEndEstimateRange(sctx, 1, debugTraceUniquePoint) + } + continue + } + var cnt float64 + cnt, err = equalRowCountOnColumn(sctx, c, lowVal, lowEncoded, realtimeRowCount, modifyCount) + if err != nil { + return 0, errors.Trace(err) + } + // If the current table row count has changed, we should scale the row count accordingly. + cnt *= c.GetIncreaseFactor(realtimeRowCount) + rowCount += cnt + if debugTrace { + debugTraceEndEstimateRange(sctx, cnt, debugTracePoint) + } + } + continue + } + // In stats ver 1, we use CM Sketch to estimate row count for point condition, which is more accurate. + // So for the small range, we convert it to points. + if c.StatsVer < 2 { + rangeVals := statistics.EnumRangeValues(lowVal, highVal, rg.LowExclude, rg.HighExclude) + + // case 2: it's a small range && using ver1 stats + if rangeVals != nil { + for _, val := range rangeVals { + cnt, err := equalRowCountOnColumn(sctx, c, val, lowEncoded, realtimeRowCount, modifyCount) + if err != nil { + return 0, err + } + // If the current table row count has changed, we should scale the row count accordingly. + cnt *= c.GetIncreaseFactor(realtimeRowCount) + if debugTrace { + debugTraceEndEstimateRange(sctx, cnt, debugTraceVer1SmallRange) + } + rowCount += cnt + } + + continue + } + } + + // case 3: it's an interval + cnt := betweenRowCountOnColumn(sctx, c, lowVal, highVal, lowEncoded, highEncoded) + // `betweenRowCount` returns count for [l, h) range, we adjust cnt for boundaries here. + // Note that, `cnt` does not include null values, we need specially handle cases + // where null is the lower bound. + // And because we use (2, MaxValue] to represent expressions like a > 2 and use [MinNotNull, 3) to represent + // expressions like b < 3, we need to exclude the special values. + if rg.LowExclude && !lowVal.IsNull() && lowVal.Kind() != types.KindMaxValue && lowVal.Kind() != types.KindMinNotNull { + lowCnt, err := equalRowCountOnColumn(sctx, c, lowVal, lowEncoded, realtimeRowCount, modifyCount) + if err != nil { + return 0, errors.Trace(err) + } + cnt -= lowCnt + cnt = mathutil.Clamp(cnt, 0, c.NotNullCount()) + } + if !rg.LowExclude && lowVal.IsNull() { + cnt += float64(c.NullCount) + } + if !rg.HighExclude && highVal.Kind() != types.KindMaxValue && highVal.Kind() != types.KindMinNotNull { + highCnt, err := equalRowCountOnColumn(sctx, c, highVal, highEncoded, realtimeRowCount, modifyCount) + if err != nil { + return 0, errors.Trace(err) + } + cnt += highCnt + } + + cnt = mathutil.Clamp(cnt, 0, c.TotalRowCount()) + + // If the current table row count has changed, we should scale the row count accordingly. + increaseFactor := c.GetIncreaseFactor(realtimeRowCount) + cnt *= increaseFactor + + // handling the out-of-range part + if (c.OutOfRange(lowVal) && !lowVal.IsNull()) || c.OutOfRange(highVal) { + histNDV := c.NDV + // Exclude the TopN + if c.StatsVer == statistics.Version2 { + histNDV -= int64(c.TopN.Num()) + } + cnt += c.Histogram.OutOfRangeRowCount(sctx, &lowVal, &highVal, modifyCount, histNDV, increaseFactor) + } + + if debugTrace { + debugTraceEndEstimateRange(sctx, cnt, debugTraceRange) + } + rowCount += cnt + } + allowZeroEst := fixcontrol.GetBoolWithDefault( + sctx.GetSessionVars().GetOptimizerFixControlMap(), + fixcontrol.Fix47400, + false, + ) + if allowZeroEst { + rowCount = mathutil.Clamp(rowCount, 0, float64(realtimeRowCount)) + } else { + // Don't allow the final result to go below 1 row + rowCount = mathutil.Clamp(rowCount, 1, float64(realtimeRowCount)) + } + return rowCount, nil +} + +// betweenRowCountOnColumn estimates the row count for interval [l, r). +func betweenRowCountOnColumn(sctx context.PlanContext, c *statistics.Column, l, r types.Datum, lowEncoded, highEncoded []byte) float64 { + histBetweenCnt := c.Histogram.BetweenRowCount(sctx, l, r) + if c.StatsVer <= statistics.Version1 { + return histBetweenCnt + } + return float64(c.TopN.BetweenCount(sctx, lowEncoded, highEncoded)) + histBetweenCnt +} + +// functions below are mainly for testing. + +// ColumnGreaterRowCount estimates the row count where the column greater than value. +func ColumnGreaterRowCount(sctx context.PlanContext, t *statistics.Table, value types.Datum, colID int64) float64 { + c := t.GetCol(colID) + if statistics.ColumnStatsIsInvalid(c, sctx, &t.HistColl, colID) { + return float64(t.RealtimeCount) / pseudoLessRate + } + return c.GreaterRowCount(value) * c.GetIncreaseFactor(t.RealtimeCount) +} + +// columnLessRowCount estimates the row count where the column less than value. Note that null values are not counted. +func columnLessRowCount(sctx context.PlanContext, t *statistics.Table, value types.Datum, colID int64) float64 { + c := t.GetCol(colID) + if statistics.ColumnStatsIsInvalid(c, sctx, &t.HistColl, colID) { + return float64(t.RealtimeCount) / pseudoLessRate + } + return c.LessRowCount(sctx, value) * c.GetIncreaseFactor(t.RealtimeCount) +} + +// columnBetweenRowCount estimates the row count where column greater or equal to a and less than b. +func columnBetweenRowCount(sctx context.PlanContext, t *statistics.Table, a, b types.Datum, colID int64) (float64, error) { + sc := sctx.GetSessionVars().StmtCtx + c := t.GetCol(colID) + if statistics.ColumnStatsIsInvalid(c, sctx, &t.HistColl, colID) { + return float64(t.RealtimeCount) / pseudoBetweenRate, nil + } + aEncoded, err := codec.EncodeKey(sc.TimeZone(), nil, a) + err = sc.HandleError(err) + if err != nil { + return 0, err + } + bEncoded, err := codec.EncodeKey(sc.TimeZone(), nil, b) + err = sc.HandleError(err) + if err != nil { + return 0, err + } + count := betweenRowCountOnColumn(sctx, c, a, b, aEncoded, bEncoded) + if a.IsNull() { + count += float64(c.NullCount) + } + return count * c.GetIncreaseFactor(t.RealtimeCount), nil +} + +// ColumnEqualRowCount estimates the row count where the column equals to value. +func ColumnEqualRowCount(sctx context.PlanContext, t *statistics.Table, value types.Datum, colID int64) (float64, error) { + c := t.GetCol(colID) + if statistics.ColumnStatsIsInvalid(c, sctx, &t.HistColl, colID) { + return float64(t.RealtimeCount) / pseudoEqualRate, nil + } + encodedVal, err := codec.EncodeKey(sctx.GetSessionVars().StmtCtx.TimeZone(), nil, value) + err = sctx.GetSessionVars().StmtCtx.HandleError(err) + if err != nil { + return 0, err + } + result, err := equalRowCountOnColumn(sctx, c, value, encodedVal, t.RealtimeCount, t.ModifyCount) + result *= c.GetIncreaseFactor(t.RealtimeCount) + return result, errors.Trace(err) +} diff --git a/pkg/planner/cardinality/row_count_index.go b/pkg/planner/cardinality/row_count_index.go new file mode 100644 index 0000000000000..1b7065662d047 --- /dev/null +++ b/pkg/planner/cardinality/row_count_index.go @@ -0,0 +1,578 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package cardinality + +import ( + "bytes" + "math" + "slices" + "strings" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/planner/context" + "github.com/pingcap/tidb/pkg/planner/util/debugtrace" + "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" + "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/statistics" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/chunk" + "github.com/pingcap/tidb/pkg/util/codec" + "github.com/pingcap/tidb/pkg/util/collate" + "github.com/pingcap/tidb/pkg/util/mathutil" + "github.com/pingcap/tidb/pkg/util/ranger" +) + +// GetRowCountByIndexRanges estimates the row count by a slice of Range. +func GetRowCountByIndexRanges(sctx context.PlanContext, coll *statistics.HistColl, idxID int64, indexRanges []*ranger.Range) (result float64, err error) { + var name string + if sctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(sctx) + debugTraceGetRowCountInput(sctx, idxID, indexRanges) + defer func() { + debugtrace.RecordAnyValuesWithNames(sctx, "Name", name, "Result", result) + debugtrace.LeaveContextCommon(sctx) + }() + } + sc := sctx.GetSessionVars().StmtCtx + idx := coll.GetIdx(idxID) + colNames := make([]string, 0, 8) + if idx != nil { + if idx.Info != nil { + name = idx.Info.Name.O + for _, col := range idx.Info.Columns { + colNames = append(colNames, col.Name.O) + } + } + } + recordUsedItemStatsStatus(sctx, idx, coll.PhysicalID, idxID) + if statistics.IndexStatsIsInvalid(sctx, idx, coll, idxID) { + colsLen := -1 + if idx != nil && idx.Info.Unique { + colsLen = len(idx.Info.Columns) + } + result, err = getPseudoRowCountByIndexRanges(sc.TypeCtx(), indexRanges, float64(coll.RealtimeCount), colsLen) + if err == nil && sc.EnableOptimizerCETrace && idx != nil { + ceTraceRange(sctx, coll.PhysicalID, colNames, indexRanges, "Index Stats-Pseudo", uint64(result)) + } + return result, err + } + realtimeCnt, modifyCount := coll.GetScaledRealtimeAndModifyCnt(idx) + if sctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.RecordAnyValuesWithNames(sctx, + "Histogram NotNull Count", idx.Histogram.NotNullCount(), + "TopN total count", idx.TopN.TotalCount(), + "Increase Factor", idx.GetIncreaseFactor(realtimeCnt), + ) + } + if idx.CMSketch != nil && idx.StatsVer == statistics.Version1 { + result, err = getIndexRowCountForStatsV1(sctx, coll, idxID, indexRanges) + } else { + result, err = getIndexRowCountForStatsV2(sctx, idx, coll, indexRanges, realtimeCnt, modifyCount) + } + if sc.EnableOptimizerCETrace { + ceTraceRange(sctx, coll.PhysicalID, colNames, indexRanges, "Index Stats", uint64(result)) + } + return result, errors.Trace(err) +} + +func getIndexRowCountForStatsV1(sctx context.PlanContext, coll *statistics.HistColl, idxID int64, indexRanges []*ranger.Range) (float64, error) { + sc := sctx.GetSessionVars().StmtCtx + debugTrace := sc.EnableOptimizerDebugTrace + if debugTrace { + debugtrace.EnterContextCommon(sctx) + defer debugtrace.LeaveContextCommon(sctx) + } + idx := coll.GetIdx(idxID) + totalCount := float64(0) + for _, ran := range indexRanges { + if debugTrace { + debugTraceStartEstimateRange(sctx, ran, nil, nil, totalCount) + } + rangePosition := getOrdinalOfRangeCond(sc, ran) + var rangeVals []types.Datum + // Try to enum the last range values. + if rangePosition != len(ran.LowVal) { + rangeVals = statistics.EnumRangeValues(ran.LowVal[rangePosition], ran.HighVal[rangePosition], ran.LowExclude, ran.HighExclude) + if rangeVals != nil { + rangePosition++ + } + } + // If first one is range, just use the previous way to estimate; if it is [NULL, NULL] range + // on single-column index, use previous way as well, because CMSketch does not contain null + // values in this case. + if rangePosition == 0 || isSingleColIdxNullRange(idx, ran) { + realtimeCnt, modifyCount := coll.GetScaledRealtimeAndModifyCnt(idx) + count, err := getIndexRowCountForStatsV2(sctx, idx, nil, []*ranger.Range{ran}, realtimeCnt, modifyCount) + if err != nil { + return 0, errors.Trace(err) + } + if debugTrace { + debugTraceEndEstimateRange(sctx, count, debugTraceRange) + } + totalCount += count + continue + } + var selectivity float64 + // use CM Sketch to estimate the equal conditions + if rangeVals == nil { + bytes, err := codec.EncodeKey(sc.TimeZone(), nil, ran.LowVal[:rangePosition]...) + err = sc.HandleError(err) + if err != nil { + return 0, errors.Trace(err) + } + selectivity, err = getEqualCondSelectivity(sctx, coll, idx, bytes, rangePosition, ran) + if err != nil { + return 0, errors.Trace(err) + } + } else { + bytes, err := codec.EncodeKey(sc.TimeZone(), nil, ran.LowVal[:rangePosition-1]...) + err = sc.HandleError(err) + if err != nil { + return 0, errors.Trace(err) + } + prefixLen := len(bytes) + for _, val := range rangeVals { + bytes = bytes[:prefixLen] + bytes, err = codec.EncodeKey(sc.TimeZone(), bytes, val) + err = sc.HandleError(err) + if err != nil { + return 0, err + } + res, err := getEqualCondSelectivity(sctx, coll, idx, bytes, rangePosition, ran) + if err != nil { + return 0, errors.Trace(err) + } + selectivity += res + } + } + // use histogram to estimate the range condition + if rangePosition != len(ran.LowVal) { + rang := ranger.Range{ + LowVal: []types.Datum{ran.LowVal[rangePosition]}, + LowExclude: ran.LowExclude, + HighVal: []types.Datum{ran.HighVal[rangePosition]}, + HighExclude: ran.HighExclude, + Collators: []collate.Collator{ran.Collators[rangePosition]}, + } + var count float64 + var err error + colUniqueIDs := coll.Idx2ColUniqueIDs[idxID] + var colUniqueID int64 + if rangePosition >= len(colUniqueIDs) { + colUniqueID = -1 + } else { + colUniqueID = colUniqueIDs[rangePosition] + } + // prefer index stats over column stats + if idxIDs, ok := coll.ColUniqueID2IdxIDs[colUniqueID]; ok && len(idxIDs) > 0 { + idxID := idxIDs[0] + count, err = GetRowCountByIndexRanges(sctx, coll, idxID, []*ranger.Range{&rang}) + } else { + count, err = GetRowCountByColumnRanges(sctx, coll, colUniqueID, []*ranger.Range{&rang}) + } + if err != nil { + return 0, errors.Trace(err) + } + selectivity = selectivity * count / idx.TotalRowCount() + } + count := selectivity * idx.TotalRowCount() + if debugTrace { + debugTraceEndEstimateRange(sctx, count, debugTraceRange) + } + totalCount += count + } + if totalCount > idx.TotalRowCount() { + totalCount = idx.TotalRowCount() + } + return totalCount, nil +} + +// isSingleColIdxNullRange checks if a range is [NULL, NULL] on a single-column index. +func isSingleColIdxNullRange(idx *statistics.Index, ran *ranger.Range) bool { + if len(idx.Info.Columns) > 1 { + return false + } + l, h := ran.LowVal[0], ran.HighVal[0] + if l.IsNull() && h.IsNull() { + return true + } + return false +} + +// It uses the modifyCount to adjust the influence of modifications on the table. +func getIndexRowCountForStatsV2(sctx context.PlanContext, idx *statistics.Index, coll *statistics.HistColl, indexRanges []*ranger.Range, realtimeRowCount, modifyCount int64) (float64, error) { + sc := sctx.GetSessionVars().StmtCtx + debugTrace := sc.EnableOptimizerDebugTrace + if debugTrace { + debugtrace.EnterContextCommon(sctx) + defer debugtrace.LeaveContextCommon(sctx) + } + totalCount := float64(0) + isSingleColIdx := len(idx.Info.Columns) == 1 + for _, indexRange := range indexRanges { + var count float64 + lb, err := codec.EncodeKey(sc.TimeZone(), nil, indexRange.LowVal...) + err = sc.HandleError(err) + if err != nil { + return 0, err + } + rb, err := codec.EncodeKey(sc.TimeZone(), nil, indexRange.HighVal...) + err = sc.HandleError(err) + if err != nil { + return 0, err + } + if debugTrace { + debugTraceStartEstimateRange(sctx, indexRange, lb, rb, totalCount) + } + fullLen := len(indexRange.LowVal) == len(indexRange.HighVal) && len(indexRange.LowVal) == len(idx.Info.Columns) + if bytes.Equal(lb, rb) { + // case 1: it's a point + if indexRange.LowExclude || indexRange.HighExclude { + if debugTrace { + debugTraceEndEstimateRange(sctx, 0, debugTraceImpossible) + } + continue + } + if fullLen { + // At most 1 in this case. + if idx.Info.Unique { + totalCount++ + if debugTrace { + debugTraceEndEstimateRange(sctx, 1, debugTraceUniquePoint) + } + continue + } + count = equalRowCountOnIndex(sctx, idx, lb, realtimeRowCount, modifyCount) + // If the current table row count has changed, we should scale the row count accordingly. + count *= idx.GetIncreaseFactor(realtimeRowCount) + if debugTrace { + debugTraceEndEstimateRange(sctx, count, debugTracePoint) + } + totalCount += count + continue + } + } + + // case 2: it's an interval + // The final interval is [low, high) + if indexRange.LowExclude { + lb = kv.Key(lb).PrefixNext() + } + if !indexRange.HighExclude { + rb = kv.Key(rb).PrefixNext() + } + l := types.NewBytesDatum(lb) + r := types.NewBytesDatum(rb) + lowIsNull := bytes.Equal(lb, nullKeyBytes) + if isSingleColIdx && lowIsNull { + count += float64(idx.Histogram.NullCount) + } + expBackoffSuccess := false + // Due to the limitation of calcFraction and convertDatumToScalar, the histogram actually won't estimate anything. + // If the first column's range is point. + if rangePosition := getOrdinalOfRangeCond(sc, indexRange); rangePosition > 0 && idx.StatsVer >= statistics.Version2 && coll != nil { + var expBackoffSel float64 + expBackoffSel, expBackoffSuccess, err = expBackoffEstimation(sctx, idx, coll, indexRange) + if err != nil { + return 0, err + } + if expBackoffSuccess { + expBackoffCnt := expBackoffSel * idx.TotalRowCount() + + upperLimit := expBackoffCnt + // Use the multi-column stats to calculate the max possible row count of [l, r) + if idx.Histogram.Len() > 0 { + _, lowerBkt, _, _ := idx.Histogram.LocateBucket(sctx, l) + _, upperBkt, _, _ := idx.Histogram.LocateBucket(sctx, r) + if debugTrace { + statistics.DebugTraceBuckets(sctx, &idx.Histogram, []int{lowerBkt - 1, upperBkt}) + } + // Use Count of the Bucket before l as the lower bound. + preCount := float64(0) + if lowerBkt > 0 { + preCount = float64(idx.Histogram.Buckets[lowerBkt-1].Count) + } + // Use Count of the Bucket where r exists as the upper bound. + upperCnt := float64(idx.Histogram.Buckets[upperBkt].Count) + upperLimit = upperCnt - preCount + upperLimit += float64(idx.TopN.BetweenCount(sctx, lb, rb)) + } + + // If the result of exponential backoff strategy is larger than the result from multi-column stats, + // use the upper limit from multi-column histogram instead. + if expBackoffCnt > upperLimit { + expBackoffCnt = upperLimit + } + count += expBackoffCnt + } + } + if !expBackoffSuccess { + count += betweenRowCountOnIndex(sctx, idx, l, r) + } + + // If the current table row count has changed, we should scale the row count accordingly. + increaseFactor := idx.GetIncreaseFactor(realtimeRowCount) + count *= increaseFactor + + // handling the out-of-range part + if (outOfRangeOnIndex(idx, l) && !(isSingleColIdx && lowIsNull)) || outOfRangeOnIndex(idx, r) { + histNDV := idx.NDV + // Exclude the TopN in Stats Version 2 + if idx.StatsVer == statistics.Version2 { + c := coll.GetCol(idx.Histogram.ID) + // If this is single column of a multi-column index - use the column's NDV rather than index NDV + isSingleColRange := len(indexRange.LowVal) == len(indexRange.HighVal) && len(indexRange.LowVal) == 1 + if isSingleColRange && !isSingleColIdx && c != nil && c.Histogram.NDV > 0 { + histNDV = c.Histogram.NDV - int64(c.TopN.Num()) + } else { + histNDV -= int64(idx.TopN.Num()) + } + } + count += idx.Histogram.OutOfRangeRowCount(sctx, &l, &r, modifyCount, histNDV, increaseFactor) + } + + if debugTrace { + debugTraceEndEstimateRange(sctx, count, debugTraceRange) + } + totalCount += count + } + allowZeroEst := fixcontrol.GetBoolWithDefault( + sctx.GetSessionVars().GetOptimizerFixControlMap(), + fixcontrol.Fix47400, + false, + ) + if allowZeroEst { + totalCount = mathutil.Clamp(totalCount, 0, float64(realtimeRowCount)) + } else { + // Don't allow the final result to go below 1 row + totalCount = mathutil.Clamp(totalCount, 1, float64(realtimeRowCount)) + } + return totalCount, nil +} + +var nullKeyBytes, _ = codec.EncodeKey(time.UTC, nil, types.NewDatum(nil)) + +func equalRowCountOnIndex(sctx context.PlanContext, idx *statistics.Index, b []byte, realtimeRowCount, modifyCount int64) (result float64) { + if sctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(sctx) + debugtrace.RecordAnyValuesWithNames(sctx, "Encoded Value", b) + defer func() { + debugtrace.RecordAnyValuesWithNames(sctx, "Result", result) + debugtrace.LeaveContextCommon(sctx) + }() + } + if len(idx.Info.Columns) == 1 { + if bytes.Equal(b, nullKeyBytes) { + return float64(idx.Histogram.NullCount) + } + } + val := types.NewBytesDatum(b) + if idx.StatsVer < statistics.Version2 { + if idx.Histogram.NDV > 0 && outOfRangeOnIndex(idx, val) { + return outOfRangeEQSelectivity(sctx, idx.Histogram.NDV, realtimeRowCount, int64(idx.TotalRowCount())) * idx.TotalRowCount() + } + if idx.CMSketch != nil { + return float64(idx.QueryBytes(sctx, b)) + } + histRowCount, _ := idx.Histogram.EqualRowCount(sctx, val, false) + return histRowCount + } + // stats version == 2 + // 1. try to find this value in TopN + if idx.TopN != nil { + count, found := idx.TopN.QueryTopN(sctx, b) + if found { + return float64(count) + } + } + // 2. try to find this value in bucket.Repeat(the last value in every bucket) + histCnt, matched := idx.Histogram.EqualRowCount(sctx, val, true) + if matched { + return histCnt + } + // 3. use uniform distribution assumption for the rest (even when this value is not covered by the range of stats) + histNDV := float64(idx.Histogram.NDV - int64(idx.TopN.Num())) + if histNDV <= 0 { + // If the table hasn't been modified, it's safe to return 0. Otherwise, the TopN could be stale - return 1. + if modifyCount == 0 { + return 0 + } + return 1 + } + return idx.Histogram.NotNullCount() / histNDV +} + +// expBackoffEstimation estimate the multi-col cases following the Exponential Backoff. See comment below for details. +func expBackoffEstimation(sctx context.PlanContext, idx *statistics.Index, coll *statistics.HistColl, indexRange *ranger.Range) (sel float64, success bool, err error) { + if sctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(sctx) + defer func() { + debugtrace.RecordAnyValuesWithNames(sctx, + "Result", sel, + "Success", success, + "error", err, + ) + debugtrace.LeaveContextCommon(sctx) + }() + } + tmpRan := []*ranger.Range{ + { + LowVal: make([]types.Datum, 1), + HighVal: make([]types.Datum, 1), + Collators: make([]collate.Collator, 1), + }, + } + colsIDs := coll.Idx2ColUniqueIDs[idx.Histogram.ID] + singleColumnEstResults := make([]float64, 0, len(indexRange.LowVal)) + // The following codes uses Exponential Backoff to reduce the impact of independent assumption. It works like: + // 1. Calc the selectivity of each column. + // 2. Sort them and choose the first 4 most selective filter and the corresponding selectivity is sel_1, sel_2, sel_3, sel_4 where i < j => sel_i < sel_j. + // 3. The final selectivity would be sel_1 * sel_2^{1/2} * sel_3^{1/4} * sel_4^{1/8}. + // This calculation reduced the independence assumption and can work well better than it. + for i := 0; i < len(indexRange.LowVal); i++ { + tmpRan[0].LowVal[0] = indexRange.LowVal[i] + tmpRan[0].HighVal[0] = indexRange.HighVal[i] + tmpRan[0].Collators[0] = indexRange.Collators[0] + if i == len(indexRange.LowVal)-1 { + tmpRan[0].LowExclude = indexRange.LowExclude + tmpRan[0].HighExclude = indexRange.HighExclude + } + colID := colsIDs[i] + var ( + count float64 + selectivity float64 + err error + foundStats bool + ) + if !statistics.ColumnStatsIsInvalid(coll.GetCol(colID), sctx, coll, colID) { + foundStats = true + count, err = GetRowCountByColumnRanges(sctx, coll, colID, tmpRan) + selectivity = count / float64(coll.RealtimeCount) + } + if idxIDs, ok := coll.ColUniqueID2IdxIDs[colID]; ok && !foundStats && len(indexRange.LowVal) > 1 { + // Note the `len(indexRange.LowVal) > 1` condition here, it means we only recursively call + // `GetRowCountByIndexRanges()` when the input `indexRange` is a multi-column range. This + // check avoids infinite recursion. + for _, idxID := range idxIDs { + if idxID == idx.Histogram.ID { + continue + } + idxStats := coll.GetIdx(idxID) + if idxStats == nil || statistics.IndexStatsIsInvalid(sctx, idxStats, coll, idxID) { + continue + } + foundStats = true + count, err = GetRowCountByIndexRanges(sctx, coll, idxID, tmpRan) + if err == nil { + break + } + realtimeCnt, _ := coll.GetScaledRealtimeAndModifyCnt(idxStats) + selectivity = count / float64(realtimeCnt) + } + } + if !foundStats { + continue + } + if err != nil { + return 0, false, err + } + singleColumnEstResults = append(singleColumnEstResults, selectivity) + } + // Sort them. + slices.Sort(singleColumnEstResults) + l := len(singleColumnEstResults) + failpoint.Inject("cleanEstResults", func() { + singleColumnEstResults = singleColumnEstResults[:0] + l = 0 + }) + if l == 1 { + return singleColumnEstResults[0], true, nil + } else if l == 0 { + return 0, false, nil + } + // Do not allow the exponential backoff to go below the available index bound. If the number of predicates + // is less than the number of index columns - use 90% of the bound to differentiate a subset from full index match. + // If there is an individual column selectivity that goes below this bound, use that selectivity only. + histNDV := coll.RealtimeCount + if idx.NDV > 0 { + histNDV = idx.NDV + } + idxLowBound := 1 / float64(min(histNDV, coll.RealtimeCount)) + if l < len(idx.Info.Columns) { + idxLowBound /= 0.9 + } + minTwoCol := min(singleColumnEstResults[0], singleColumnEstResults[1], idxLowBound) + multTwoCol := singleColumnEstResults[0] * math.Sqrt(singleColumnEstResults[1]) + if l == 2 { + return max(minTwoCol, multTwoCol), true, nil + } + minThreeCol := min(minTwoCol, singleColumnEstResults[2]) + multThreeCol := multTwoCol * math.Sqrt(math.Sqrt(singleColumnEstResults[2])) + if l == 3 { + return max(minThreeCol, multThreeCol), true, nil + } + minFourCol := min(minThreeCol, singleColumnEstResults[3]) + multFourCol := multThreeCol * math.Sqrt(math.Sqrt(math.Sqrt(singleColumnEstResults[3]))) + return max(minFourCol, multFourCol), true, nil +} + +// outOfRangeOnIndex checks if the datum is out of the range. +func outOfRangeOnIndex(idx *statistics.Index, val types.Datum) bool { + if !idx.Histogram.OutOfRange(val) { + return false + } + if idx.Histogram.Len() > 0 && matchPrefix(idx.Histogram.Bounds.GetRow(0), 0, &val) { + return false + } + return true +} + +// matchPrefix checks whether ad is the prefix of value +func matchPrefix(row chunk.Row, colIdx int, ad *types.Datum) bool { + switch ad.Kind() { + case types.KindString, types.KindBytes, types.KindBinaryLiteral, types.KindMysqlBit: + return strings.HasPrefix(row.GetString(colIdx), ad.GetString()) + } + return false +} + +// betweenRowCountOnIndex estimates the row count for interval [l, r). +// The input sctx is just for debug trace, you can pass nil safely if that's not needed. +func betweenRowCountOnIndex(sctx context.PlanContext, idx *statistics.Index, l, r types.Datum) float64 { + histBetweenCnt := idx.Histogram.BetweenRowCount(sctx, l, r) + if idx.StatsVer == statistics.Version1 { + return histBetweenCnt + } + return float64(idx.TopN.BetweenCount(sctx, l.GetBytes(), r.GetBytes())) + histBetweenCnt +} + +// getOrdinalOfRangeCond gets the ordinal of the position range condition, +// if not exist, it returns the end position. +func getOrdinalOfRangeCond(sc *stmtctx.StatementContext, ran *ranger.Range) int { + for i := range ran.LowVal { + a, b := ran.LowVal[i], ran.HighVal[i] + cmp, err := a.Compare(sc.TypeCtx(), &b, ran.Collators[0]) + if err != nil { + return 0 + } + if cmp != 0 { + return i + } + } + return len(ran.LowVal) +} diff --git a/pkg/planner/cardinality/testdata/cardinality_suite_out.json b/pkg/planner/cardinality/testdata/cardinality_suite_out.json new file mode 100644 index 0000000000000..cfae3231c3f52 --- /dev/null +++ b/pkg/planner/cardinality/testdata/cardinality_suite_out.json @@ -0,0 +1,4194 @@ +[ + { + "Name": "TestCollationColumnEstimate", + "Cases": [ + [ + "test t a 0 \u0000A\u0000A\u0000A 2", + "test t a 0 \u0000B\u0000B\u0000B 2" + ], + [ + "TableReader_7 2.00 root data:Selection_6", + "└─Selection_6 2.00 cop[tikv] eq(test.t.a, \"aÄa\")", + " └─TableFullScan_5 4.00 cop[tikv] table:t keep order:false" + ], + [ + "TableReader_7 2.00 root data:Selection_6", + "└─Selection_6 2.00 cop[tikv] gt(test.t.a, \"aÄa\")", + " └─TableFullScan_5 4.00 cop[tikv] table:t keep order:false" + ] + ] + }, + { + "Name": "TestOutOfRangeEstimation", + "Cases": [ + { + "Start": 800, + "End": 900, + "Count": 791.004166655054 + }, + { + "Start": 900, + "End": 950, + "Count": 247.04782734719248 + }, + { + "Start": 950, + "End": 1000, + "Count": 226.14487557169574 + }, + { + "Start": 1000, + "End": 1050, + "Count": 205.24192379619902 + }, + { + "Start": 1050, + "End": 1100, + "Count": 184.33897202070227 + }, + { + "Start": 1150, + "End": 1200, + "Count": 142.53306846970884 + }, + { + "Start": 1200, + "End": 1300, + "Count": 214.85728161292752 + }, + { + "Start": 1300, + "End": 1400, + "Count": 131.2454745109406 + }, + { + "Start": 1400, + "End": 1500, + "Count": 47.650389770374105 + }, + { + "Start": 1500, + "End": 1600, + "Count": 7.5 + }, + { + "Start": 300, + "End": 899, + "Count": 4500 + }, + { + "Start": 800, + "End": 1000, + "Count": 1249.196869573942 + }, + { + "Start": 900, + "End": 1500, + "Count": 1502.495833344946 + }, + { + "Start": 300, + "End": 1500, + "Count": 4500 + }, + { + "Start": 200, + "End": 300, + "Count": 466.52882098990807 + }, + { + "Start": 100, + "End": 200, + "Count": 382.91701388792114 + }, + { + "Start": 200, + "End": 400, + "Count": 1188.7788209899081 + }, + { + "Start": 200, + "End": 1000, + "Count": 4500 + }, + { + "Start": 0, + "End": 100, + "Count": 299.3052067859343 + }, + { + "Start": -100, + "End": 100, + "Count": 299.3052067859343 + }, + { + "Start": -100, + "End": 0, + "Count": 7.5 + } + ] + }, + { + "Name": "TestOutOfRangeEstimationAfterDelete", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t where a <= 300", + "Result": [ + "TableReader 1003.33 root data:Selection", + "└─Selection 1003.33 cop[tikv] le(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a < 300", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] lt(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a <= 500", + "Result": [ + "TableReader 1670.00 root data:Selection", + "└─Selection 1670.00 cop[tikv] le(test.t.a, 500)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a >= 300 and a <= 900", + "Result": [ + "TableReader 2000.00 root data:Selection", + "└─Selection 2000.00 cop[tikv] ge(test.t.a, 300), le(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a >= 900", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] ge(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 900", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] gt(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a >= 300", + "Result": [ + "TableReader 2000.00 root data:Selection", + "└─Selection 2000.00 cop[tikv] ge(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a <= 900", + "Result": [ + "TableReader 2000.00 root data:Selection", + "└─Selection 2000.00 cop[tikv] le(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 800 and a < 1000", + "Result": [ + "TableReader 793.20 root data:Selection", + "└─Selection 793.20 cop[tikv] gt(test.t.a, 800), lt(test.t.a, 1000)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 900 and a < 1000", + "Result": [ + "TableReader 458.19 root data:Selection", + "└─Selection 458.19 cop[tikv] gt(test.t.a, 900), lt(test.t.a, 1000)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 900 and a < 1100", + "Result": [ + "TableReader 832.77 root data:Selection", + "└─Selection 832.77 cop[tikv] gt(test.t.a, 900), lt(test.t.a, 1100)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 200 and a < 300", + "Result": [ + "TableReader 459.03 root data:Selection", + "└─Selection 459.03 cop[tikv] gt(test.t.a, 200), lt(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 100 and a < 300", + "Result": [ + "TableReader 834.45 root data:Selection", + "└─Selection 834.45 cop[tikv] gt(test.t.a, 100), lt(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "set @@tidb_opt_objective = 'determinate'", + "Result": null + }, + { + "SQL": "explain format = 'brief' select * from t where a <= 300", + "Result": [ + "TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] le(test.t.a, 300)", + " └─TableFullScan 3000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a <= 500", + "Result": [ + "TableReader 1010.00 root data:Selection", + "└─Selection 1010.00 cop[tikv] le(test.t.a, 500)", + " └─TableFullScan 3000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 900", + "Result": [ + "TableReader 5.00 root data:Selection", + "└─Selection 5.00 cop[tikv] gt(test.t.a, 900)", + " └─TableFullScan 3000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a <= 900", + "Result": [ + "TableReader 3000.00 root data:Selection", + "└─Selection 3000.00 cop[tikv] le(test.t.a, 900)", + " └─TableFullScan 3000.00 cop[tikv] table:t keep order:false" + ] + } + ] + }, + { + "Name": "TestColumnIndexNullEstimation", + "Cases": [ + [ + "IndexReader_6 4.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 4.00 cop[tikv] table:t, index:idx_b(b) range:[NULL,NULL], keep order:false" + ], + [ + "IndexReader_6 1.00 root index:IndexFullScan_5", + "└─IndexFullScan_5 1.00 cop[tikv] table:t, index:idx_b(b) keep order:false" + ], + [ + "IndexReader_6 4.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 4.00 cop[tikv] table:t, index:idx_b(b) range:[NULL,NULL], (3,+inf], keep order:false" + ], + [ + "IndexReader_5 5.00 root index:IndexFullScan_4", + "└─IndexFullScan_4 5.00 cop[tikv] table:t, index:idx_b(b) keep order:false" + ], + [ + "IndexReader_6 1.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 1.00 cop[tikv] table:t, index:idx_b(b) range:[-inf,4), keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] isnull(test.t.a)", + " └─TableFullScan_5 5.00 cop[tikv] table:t keep order:false" + ], + [ + "TableReader_7 4.00 root data:Selection_6", + "└─Selection_6 4.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan_5 5.00 cop[tikv] table:t keep order:false" + ], + [ + "TableReader_7 2.00 root data:Selection_6", + "└─Selection_6 2.00 cop[tikv] or(isnull(test.t.a), gt(test.t.a, 3))", + " └─TableFullScan_5 5.00 cop[tikv] table:t keep order:false" + ], + [ + "TableReader_5 5.00 root data:TableFullScan_4", + "└─TableFullScan_4 5.00 cop[tikv] table:t keep order:false" + ], + [ + "TableReader_7 3.00 root data:Selection_6", + "└─Selection_6 3.00 cop[tikv] lt(test.t.a, 4)", + " └─TableFullScan_5 5.00 cop[tikv] table:t keep order:false" + ] + ] + }, + { + "Name": "TestUniqCompEqualEst", + "Cases": [ + [ + "Point_Get_5 1.00 root table:t, clustered index:PRIMARY(a, b) " + ] + ] + }, + { + "Name": "TestDNFCondSelectivity", + "Cases": [ + { + "SQL": "select * from t where b > 7 or c < 4", + "Selectivity": 0.34375 + }, + { + "SQL": "select * from t where d < 5 or b > 6", + "Selectivity": 0.625 + }, + { + "SQL": "select * from t where a > 8 or d < 4 or c > 7 or b < 5", + "Selectivity": 0.82421875 + }, + { + "SQL": "select * from t where a < 8 and (b > 10 or c < 3 or b > 4) and a > 2", + "Selectivity": 0.3125 + } + ] + }, + { + "Name": "TestSmallRangeEstimation", + "Cases": [ + { + "Start": 5, + "End": 5, + "Count": 3 + }, + { + "Start": 5, + "End": 6, + "Count": 6 + }, + { + "Start": 5, + "End": 10, + "Count": 18 + }, + { + "Start": 5, + "End": 15, + "Count": 33 + }, + { + "Start": 10, + "End": 15, + "Count": 18 + }, + { + "Start": 5, + "End": 25, + "Count": 63 + }, + { + "Start": 25, + "End": 25, + "Count": 3 + } + ] + }, + { + "Name": "TestTopNAssistedEstimationWithoutNewCollation", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t where a like '%111%'", + "Result": [ + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.a, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a not like '%111%'", + "Result": [ + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.a, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a regexp '.*111.*'", + "Result": [ + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] regexp(test.t.a, \".*111.*\")", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a not regexp '.*111.*'", + "Result": [ + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(istrue_with_null(regexp(test.t.a, \".*111.*\")))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where ifnull(a, '1111') like '%111%'", + "Result": [ + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.a, \"1111\"), \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where f like '%111%'", + "Result": [ + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.f, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where f not like '%111%'", + "Result": [ + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.f, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where f regexp '.*111.*'", + "Result": [ + "Selection 32.00 root regexp(test.t.f, \".*111.*\")", + "└─TableReader 40.00 root data:TableFullScan", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where f not regexp '.*111.*'", + "Result": [ + "Selection 32.00 root not(regexp(test.t.f, \".*111.*\"))", + "└─TableReader 40.00 root data:TableFullScan", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where ifnull(f, '1111') like '%111%'", + "Result": [ + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.f, \"1111\"), \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a like '%111%' and f rlike '.*111.*'", + "Result": [ + "Selection 24.00 root regexp(test.t.f, \".*111.*\")", + "└─TableReader 30.00 root data:Selection", + " └─Selection 30.00 cop[tikv] like(test.t.a, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a like '%111%' or f like '%111%'", + "Result": [ + "TableReader 37.50 root data:Selection", + "└─Selection 37.50 cop[tikv] or(like(test.t.a, \"%111%\", 92), like(test.t.f, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%111%'", + "Result": [ + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.b, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not like '%111%'", + "Result": [ + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.b, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b regexp '.*111.*'", + "Result": [ + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] regexp(test.t.b, \".*111.*\")", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not regexp '.*111.*'", + "Result": [ + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(istrue_with_null(regexp(test.t.b, \".*111.*\")))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where ifnull(b, '1111') like '%111%'", + "Result": [ + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.b, \"1111\"), \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%111%' and f like '%111%'", + "Result": [ + "TableReader 22.50 root data:Selection", + "└─Selection 22.50 cop[tikv] like(test.t.b, \"%111%\", 92), like(test.t.f, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%111%' or f like '%111%'", + "Result": [ + "TableReader 37.50 root data:Selection", + "└─Selection 37.50 cop[tikv] or(like(test.t.b, \"%111%\", 92), like(test.t.f, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where c like '%111%'", + "Result": [ + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.c, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where c not like '%111%'", + "Result": [ + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.c, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where ifnull(c, '1111') like '%111%'", + "Result": [ + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.c, \"1111\"), \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where d like '%111%'", + "Result": [ + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.d, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where d not like '%111%'", + "Result": [ + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.d, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where ifnull(c, '1111') like '%111%'", + "Result": [ + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.c, \"1111\"), \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where e like '%111%'", + "Result": [ + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.e, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where e not like '%111%'", + "Result": [ + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.e, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where ifnull(e, '1111') like '%111%'", + "Result": [ + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.e, \"1111\"), \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + } + ] + }, + { + "Name": "TestTopNAssistedEstimationWithNewCollation", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t where a like '%111%'", + "Result": [ + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.a, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a not like '%111%'", + "Result": [ + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.a, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a regexp '.*111.*'", + "Result": [ + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] regexp(test.t.a, \".*111.*\")", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a not regexp '.*111.*'", + "Result": [ + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(istrue_with_null(regexp(test.t.a, \".*111.*\")))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where ifnull(a, '1111') like '%111%'", + "Result": [ + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.a, \"1111\"), \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where f like '%111%'", + "Result": [ + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.f, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where f not like '%111%'", + "Result": [ + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.f, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where f regexp '.*111.*'", + "Result": [ + "Selection 32.00 root regexp(test.t.f, \".*111.*\")", + "└─TableReader 40.00 root data:TableFullScan", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where f not regexp '.*111.*'", + "Result": [ + "Selection 32.00 root not(regexp(test.t.f, \".*111.*\"))", + "└─TableReader 40.00 root data:TableFullScan", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where ifnull(f, '1111') like '%111%'", + "Result": [ + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.f, \"1111\"), \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a like '%111%' and f rlike '.*111.*'", + "Result": [ + "Selection 24.00 root regexp(test.t.f, \".*111.*\")", + "└─TableReader 30.00 root data:Selection", + " └─Selection 30.00 cop[tikv] like(test.t.a, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a like '%111%' or f like '%111%'", + "Result": [ + "TableReader 37.50 root data:Selection", + "└─Selection 37.50 cop[tikv] or(like(test.t.a, \"%111%\", 92), like(test.t.f, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%111%'", + "Result": [ + "TableReader 4.00 root data:Selection", + "└─Selection 4.00 cop[tikv] like(test.t.b, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not like '%111%'", + "Result": [ + "TableReader 36.00 root data:Selection", + "└─Selection 36.00 cop[tikv] not(like(test.t.b, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b regexp '.*111.*'", + "Result": [ + "TableReader 4.00 root data:Selection", + "└─Selection 4.00 cop[tikv] regexp(test.t.b, \".*111.*\")", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not regexp '.*111.*'", + "Result": [ + "TableReader 36.00 root data:Selection", + "└─Selection 36.00 cop[tikv] not(istrue_with_null(regexp(test.t.b, \".*111.*\")))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where ifnull(b, '1111') like '%111%'", + "Result": [ + "TableReader 4.00 root data:Selection", + "└─Selection 4.00 cop[tikv] like(ifnull(test.t.b, \"1111\"), \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%111%' and f like '%111%'", + "Result": [ + "TableReader 3.00 root data:Selection", + "└─Selection 3.00 cop[tikv] like(test.t.b, \"%111%\", 92), like(test.t.f, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%111%' or f like '%111%'", + "Result": [ + "TableReader 31.00 root data:Selection", + "└─Selection 31.00 cop[tikv] or(like(test.t.b, \"%111%\", 92), like(test.t.f, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where c like '%111%'", + "Result": [ + "TableReader 4.00 root data:Selection", + "└─Selection 4.00 cop[tikv] like(test.t.c, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where c not like '%111%'", + "Result": [ + "TableReader 36.00 root data:Selection", + "└─Selection 36.00 cop[tikv] not(like(test.t.c, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where ifnull(c, '1111') like '%111%'", + "Result": [ + "TableReader 4.00 root data:Selection", + "└─Selection 4.00 cop[tikv] like(ifnull(test.t.c, \"1111\"), \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where d like '%111%'", + "Result": [ + "TableReader 4.00 root data:Selection", + "└─Selection 4.00 cop[tikv] like(test.t.d, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where d not like '%111%'", + "Result": [ + "TableReader 36.00 root data:Selection", + "└─Selection 36.00 cop[tikv] not(like(test.t.d, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where ifnull(c, '1111') like '%111%'", + "Result": [ + "TableReader 4.00 root data:Selection", + "└─Selection 4.00 cop[tikv] like(ifnull(test.t.c, \"1111\"), \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where e like '%111%'", + "Result": [ + "TableReader 4.00 root data:Selection", + "└─Selection 4.00 cop[tikv] like(test.t.e, \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where e not like '%111%'", + "Result": [ + "TableReader 36.00 root data:Selection", + "└─Selection 36.00 cop[tikv] not(like(test.t.e, \"%111%\", 92))", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where ifnull(e, '1111') like '%111%'", + "Result": [ + "TableReader 4.00 root data:Selection", + "└─Selection 4.00 cop[tikv] like(ifnull(test.t.e, \"1111\"), \"%111%\", 92)", + " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" + ] + } + ] + }, + { + "Name": "TestGlobalStatsOutOfRangeEstimationAfterDelete", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t where a <= 300", + "Result": [ + "TableReader 1003.33 root partition:p0 data:Selection", + "└─Selection 1003.33 cop[tikv] le(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a < 300", + "Result": [ + "TableReader 1000.00 root partition:p0 data:Selection", + "└─Selection 1000.00 cop[tikv] lt(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a <= 500", + "Result": [ + "TableReader 1670.00 root partition:p0,p1 data:Selection", + "└─Selection 1670.00 cop[tikv] le(test.t.a, 500)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a >= 300 and a <= 900", + "Result": [ + "TableReader 2000.00 root partition:p0,p1,p2,p3 data:Selection", + "└─Selection 2000.00 cop[tikv] ge(test.t.a, 300), le(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a >= 900", + "Result": [ + "TableReader 1000.00 root partition:p3,p4 data:Selection", + "└─Selection 1000.00 cop[tikv] ge(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 900", + "Result": [ + "TableReader 1000.00 root partition:p3,p4 data:Selection", + "└─Selection 1000.00 cop[tikv] gt(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a >= 300", + "Result": [ + "TableReader 2000.00 root partition:all data:Selection", + "└─Selection 2000.00 cop[tikv] ge(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a <= 900", + "Result": [ + "TableReader 2000.00 root partition:p0,p1,p2,p3 data:Selection", + "└─Selection 2000.00 cop[tikv] le(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 800 and a < 1000", + "Result": [ + "TableReader 793.20 root partition:p3 data:Selection", + "└─Selection 793.20 cop[tikv] gt(test.t.a, 800), lt(test.t.a, 1000)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 900 and a < 1000", + "Result": [ + "TableReader 458.19 root partition:p3 data:Selection", + "└─Selection 458.19 cop[tikv] gt(test.t.a, 900), lt(test.t.a, 1000)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 900 and a < 1100", + "Result": [ + "TableReader 832.77 root partition:p3,p4 data:Selection", + "└─Selection 832.77 cop[tikv] gt(test.t.a, 900), lt(test.t.a, 1100)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 200 and a < 300", + "Result": [ + "TableReader 459.03 root partition:p0 data:Selection", + "└─Selection 459.03 cop[tikv] gt(test.t.a, 200), lt(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 100 and a < 300", + "Result": [ + "TableReader 834.45 root partition:p0 data:Selection", + "└─Selection 834.45 cop[tikv] gt(test.t.a, 100), lt(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + } + ] + }, + { + "Name": "TestOrderingIdxSelectivityThreshold", + "Cases": [ + { + "Query": "set @@tidb_opt_ordering_index_selectivity_ratio = -1", + "Result": null + }, + { + "Query": "set @@tidb_opt_ordering_index_selectivity_threshold = 0", + "Result": null + }, + { + "Query": "explain format = 'brief' select * from t where b >= 9950 order by c limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexFullScan(Build) 200.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 1.00 cop[tikv] ge(test.t.b, 9950)", + " └─TableRowIDScan 200.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where b >= 9950 order by c desc limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexFullScan(Build) 200.00 cop[tikv] table:t, index:ic(c) keep order:true, desc, stats:pseudo", + " └─Selection(Probe) 1.00 cop[tikv] ge(test.t.b, 9950)", + " └─TableRowIDScan 200.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where b >= 8999 order by c limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexFullScan(Build) 9.99 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 1.00 cop[tikv] ge(test.t.b, 8999)", + " └─TableRowIDScan 9.99 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where b >= 9000 order by c limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexFullScan(Build) 10.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 1.00 cop[tikv] ge(test.t.b, 9000)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where b >= 9001 order by c limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexFullScan(Build) 10.01 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 1.00 cop[tikv] ge(test.t.b, 9001)", + " └─TableRowIDScan 10.01 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where a < 10001 order by c limit 1", + "Result": [ + "IndexLookUp 1.00 root limit embedded(offset:0, count:1)", + "├─Limit(Build) 1.00 cop[tikv] offset:0, count:1", + "│ └─Selection 1.00 cop[tikv] lt(test.t.a, 10001)", + "│ └─IndexFullScan 10.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + "└─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where a < 10000 order by c limit 1", + "Result": [ + "IndexLookUp 1.00 root limit embedded(offset:0, count:1)", + "├─Limit(Build) 1.00 cop[tikv] offset:0, count:1", + "│ └─Selection 1.00 cop[tikv] lt(test.t.a, 10000)", + "│ └─IndexFullScan 10.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + "└─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where a < 9999 order by c limit 1", + "Result": [ + "IndexLookUp 1.00 root limit embedded(offset:0, count:1)", + "├─Limit(Build) 1.00 cop[tikv] offset:0, count:1", + "│ └─Selection 1.00 cop[tikv] lt(test.t.a, 9999)", + "│ └─IndexFullScan 10.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + "└─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where b >= 0 and b <= 100 or c >= 0 and c <= 100 order by c limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexFullScan(Build) 49.76 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 1.00 cop[tikv] or(and(ge(test.t.b, 0), le(test.t.b, 100)), and(ge(test.t.c, 0), le(test.t.c, 100)))", + " └─TableRowIDScan 49.76 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where a < 1000 and c < 100 limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─TableReader 1.00 root data:Limit", + " └─Limit 1.00 cop[tikv] offset:0, count:1", + " └─Selection 1.00 cop[tikv] lt(test.t.c, 100)", + " └─TableRangeScan 100.00 cop[tikv] table:t range:[-inf,1000), keep order:false, stats:pseudo" + ] + }, + { + "Query": "set @@tidb_opt_ordering_index_selectivity_threshold = 0.1", + "Result": null + }, + { + "Query": "explain format = 'brief' select * from t where b >= 9950 order by c limit 1", + "Result": [ + "TopN 1.00 root test.t.c, offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexRangeScan(Build) 500.00 cop[tikv] table:t, index:ib(b) range:[9950,+inf], keep order:false, stats:pseudo", + " └─TopN(Probe) 1.00 cop[tikv] test.t.c, offset:0, count:1", + " └─TableRowIDScan 500.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where b >= 9950 order by c desc limit 1", + "Result": [ + "TopN 1.00 root test.t.c:desc, offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexRangeScan(Build) 500.00 cop[tikv] table:t, index:ib(b) range:[9950,+inf], keep order:false, stats:pseudo", + " └─TopN(Probe) 1.00 cop[tikv] test.t.c:desc, offset:0, count:1", + " └─TableRowIDScan 500.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where b >= 8999 order by c limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexFullScan(Build) 9.99 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 1.00 cop[tikv] ge(test.t.b, 8999)", + " └─TableRowIDScan 9.99 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where b >= 9000 order by c limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexFullScan(Build) 10.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 1.00 cop[tikv] ge(test.t.b, 9000)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where b >= 9001 order by c limit 1", + "Result": [ + "TopN 1.00 root test.t.c, offset:0, count:1", + "└─TableReader 1.00 root data:TopN", + " └─TopN 1.00 cop[tikv] test.t.c, offset:0, count:1", + " └─Selection 9990.00 cop[tikv] ge(test.t.b, 9001)", + " └─TableFullScan 100000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where a < 10001 order by c limit 1", + "Result": [ + "IndexLookUp 1.00 root limit embedded(offset:0, count:1)", + "├─Limit(Build) 1.00 cop[tikv] offset:0, count:1", + "│ └─Selection 1.00 cop[tikv] lt(test.t.a, 10001)", + "│ └─IndexFullScan 10.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + "└─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where a < 10000 order by c limit 1", + "Result": [ + "IndexLookUp 1.00 root limit embedded(offset:0, count:1)", + "├─Limit(Build) 1.00 cop[tikv] offset:0, count:1", + "│ └─Selection 1.00 cop[tikv] lt(test.t.a, 10000)", + "│ └─IndexFullScan 10.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + "└─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where a < 9999 order by c limit 1", + "Result": [ + "TopN 1.00 root test.t.c, offset:0, count:1", + "└─TableReader 1.00 root data:TopN", + " └─TopN 1.00 cop[tikv] test.t.c, offset:0, count:1", + " └─TableRangeScan 9999.00 cop[tikv] table:t range:[-inf,9999), keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where b >= 0 and b <= 50 or c >= 0 and c <= 50 order by c limit 1", + "Result": [ + "TopN 1.00 root test.t.c, offset:0, count:1", + "└─IndexMerge 1.00 root type: union", + " ├─IndexRangeScan(Build) 510.00 cop[tikv] table:t, index:ib(b) range:[0,50], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 510.00 cop[tikv] table:t, index:ic(c) range:[0,50], keep order:false, stats:pseudo", + " └─TopN(Probe) 1.00 cop[tikv] test.t.c, offset:0, count:1", + " └─TableRowIDScan 1017.40 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where b >= 9950 and c >= 9950 order by c limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexRangeScan(Build) 500.00 cop[tikv] table:t, index:ic(c) range:[9950,+inf], keep order:true, stats:pseudo", + " └─Selection(Probe) 1.00 cop[tikv] ge(test.t.b, 9950)", + " └─TableRowIDScan 500.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where b >= 9950 and c >= 9900 order by c limit 1", + "Result": [ + "TopN 1.00 root test.t.c, offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexRangeScan(Build) 500.00 cop[tikv] table:t, index:ib(b) range:[9950,+inf], keep order:false, stats:pseudo", + " └─TopN(Probe) 1.00 cop[tikv] test.t.c, offset:0, count:1", + " └─Selection 5.00 cop[tikv] ge(test.t.c, 9900)", + " └─TableRowIDScan 500.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where a < 1000 and c < 100 limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─TableReader 1.00 root data:Limit", + " └─Limit 1.00 cop[tikv] offset:0, count:1", + " └─Selection 1.00 cop[tikv] lt(test.t.c, 100)", + " └─TableRangeScan 100.00 cop[tikv] table:t range:[-inf,1000), keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestOrderingIdxSelectivityRatio", + "Cases": [ + { + "Query": "set @@tidb_opt_ordering_index_selectivity_threshold = 0", + "Result": null + }, + { + "Query": "set @@tidb_opt_ordering_index_selectivity_ratio = -1", + "Result": null + }, + { + "Query": "explain format = 'brief' select * from t use index (ic) where b >= 900 order by c limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexFullScan(Build) 10.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 1.00 cop[tikv] ge(test.t.b, 900)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t use index (ic) where b >= 900 order by c limit 10", + "Result": [ + "Limit 10.00 root offset:0, count:10", + "└─IndexLookUp 10.00 root ", + " ├─IndexFullScan(Build) 100.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 10.00 cop[tikv] ge(test.t.b, 900)", + " └─TableRowIDScan 100.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where a < 500 and c < 100 limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─TableReader 1.00 root data:Limit", + " └─Limit 1.00 cop[tikv] offset:0, count:1", + " └─Selection 1.00 cop[tikv] lt(test.t.c, 100)", + " └─TableRangeScan 10.00 cop[tikv] table:t range:[-inf,500), keep order:false, stats:pseudo" + ] + }, + { + "Query": "set @@tidb_opt_ordering_index_selectivity_ratio = 0", + "Result": null + }, + { + "Query": "explain format = 'brief' select * from t use index (ic) where b >= 900 order by c limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexFullScan(Build) 1.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 1.00 cop[tikv] ge(test.t.b, 900)", + " └─TableRowIDScan 1.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t use index (ic) where b >= 900 order by c limit 10", + "Result": [ + "Limit 10.00 root offset:0, count:10", + "└─IndexLookUp 10.00 root ", + " ├─IndexFullScan(Build) 10.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 10.00 cop[tikv] ge(test.t.b, 900)", + " └─TableRowIDScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where a < 500 and c < 100 limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─TableReader 1.00 root data:Limit", + " └─Limit 1.00 cop[tikv] offset:0, count:1", + " └─Selection 1.00 cop[tikv] lt(test.t.c, 100)", + " └─TableRangeScan 10.00 cop[tikv] table:t range:[-inf,500), keep order:false, stats:pseudo" + ] + }, + { + "Query": "set @@tidb_opt_ordering_index_selectivity_ratio = 0.1", + "Result": null + }, + { + "Query": "explain format = 'brief' select * from t use index (ic) where b >= 900 order by c limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexFullScan(Build) 91.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 1.00 cop[tikv] ge(test.t.b, 900)", + " └─TableRowIDScan 91.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t use index (ic) where b >= 900 order by c limit 10", + "Result": [ + "Limit 10.00 root offset:0, count:10", + "└─IndexLookUp 10.00 root ", + " ├─IndexFullScan(Build) 100.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 10.00 cop[tikv] ge(test.t.b, 900)", + " └─TableRowIDScan 100.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where a < 500 and c < 100 limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─TableReader 1.00 root data:Limit", + " └─Limit 1.00 cop[tikv] offset:0, count:1", + " └─Selection 1.00 cop[tikv] lt(test.t.c, 100)", + " └─TableRangeScan 10.00 cop[tikv] table:t range:[-inf,500), keep order:false, stats:pseudo" + ] + }, + { + "Query": "set @@tidb_opt_ordering_index_selectivity_ratio = 0.5", + "Result": null + }, + { + "Query": "explain format = 'brief' select * from t use index (ic) where b >= 900 order by c limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─IndexLookUp 1.00 root ", + " ├─IndexFullScan(Build) 451.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 1.00 cop[tikv] ge(test.t.b, 900)", + " └─TableRowIDScan 451.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t use index (ic) where b >= 900 order by c limit 10", + "Result": [ + "Limit 10.00 root offset:0, count:10", + "└─IndexLookUp 10.00 root ", + " ├─IndexFullScan(Build) 460.00 cop[tikv] table:t, index:ic(c) keep order:true, stats:pseudo", + " └─Selection(Probe) 10.00 cop[tikv] ge(test.t.b, 900)", + " └─TableRowIDScan 460.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "Query": "explain format = 'brief' select * from t where a < 500 and c < 100 limit 1", + "Result": [ + "Limit 1.00 root offset:0, count:1", + "└─TableReader 1.00 root data:Limit", + " └─Limit 1.00 cop[tikv] offset:0, count:1", + " └─Selection 1.00 cop[tikv] lt(test.t.c, 100)", + " └─TableRangeScan 10.00 cop[tikv] table:t range:[-inf,500), keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestTraceCE", + "Cases": [ + { + "Expr": "a > 0 and a < 2", + "Trace": [ + { + "table_name": "t", + "type": "Column Stats-Point", + "expr": "((a = 1))", + "row_count": 4 + }, + { + "table_name": "t", + "type": "Column Stats-Range", + "expr": "((a > 0 and a < 2))", + "row_count": 4 + }, + { + "table_name": "t", + "type": "Index Stats-Point", + "expr": "((a = 1))", + "row_count": 4 + }, + { + "table_name": "t", + "type": "Table Stats-Expression-CNF", + "expr": "`and`(`gt`(test.t.a, 0), `lt`(test.t.a, 2))", + "row_count": 4 + } + ] + }, + { + "Expr": "a >= 1 and a < 10", + "Trace": [ + { + "table_name": "t", + "type": "Column Stats-Range", + "expr": "((a >= 1 and a < 10))", + "row_count": 6 + }, + { + "table_name": "t", + "type": "Index Stats-Range", + "expr": "((a >= 1 and a < 10))", + "row_count": 6 + }, + { + "table_name": "t", + "type": "Table Stats-Expression-CNF", + "expr": "`and`(`ge`(test.t.a, 1), `lt`(test.t.a, 10))", + "row_count": 6 + } + ] + }, + { + "Expr": "a < 3 or b < 4", + "Trace": [ + { + "table_name": "t", + "type": "Column Stats-Range", + "expr": "((a < 3))", + "row_count": 6 + }, + { + "table_name": "t", + "type": "Column Stats-Range", + "expr": "((b < 4))", + "row_count": 6 + }, + { + "table_name": "t", + "type": "Index Stats-Range", + "expr": "((a < 3))", + "row_count": 6 + }, + { + "table_name": "t", + "type": "Table Stats-Expression-CNF", + "expr": "`lt`(test.t.a, 3)", + "row_count": 6 + }, + { + "table_name": "t", + "type": "Table Stats-Expression-CNF", + "expr": "`lt`(test.t.b, 4)", + "row_count": 6 + }, + { + "table_name": "t", + "type": "Table Stats-Expression-CNF", + "expr": "`or`(`lt`(test.t.a, 3), `lt`(test.t.b, 4))", + "row_count": 6 + }, + { + "table_name": "t", + "type": "Table Stats-Expression-DNF", + "expr": "`or`(`lt`(test.t.a, 3), `lt`(test.t.b, 4))", + "row_count": 6 + } + ] + }, + { + "Expr": "a = 1 and b = 2", + "Trace": [ + { + "table_name": "t", + "type": "Column Stats-Point", + "expr": "((a = 1))", + "row_count": 4 + }, + { + "table_name": "t", + "type": "Column Stats-Point", + "expr": "((b = 2))", + "row_count": 3 + }, + { + "table_name": "t", + "type": "Index Stats-Point", + "expr": "((a = 1) and (b = 2))", + "row_count": 2 + }, + { + "table_name": "t", + "type": "Table Stats-Expression-CNF", + "expr": "`and`(`eq`(test.t.a, 1), `eq`(test.t.b, 2))", + "row_count": 2 + } + ] + }, + { + "Expr": "a < 5 or a > 10", + "Trace": [ + { + "table_name": "t", + "type": "Column Stats-Range", + "expr": "((a < 5)) or ((a > 10 and true))", + "row_count": 6 + }, + { + "table_name": "t", + "type": "Index Stats-Range", + "expr": "((a < 5)) or ((a > 10 and true))", + "row_count": 6 + }, + { + "table_name": "t", + "type": "Table Stats-Expression-CNF", + "expr": "`or`(`lt`(test.t.a, 5), `gt`(test.t.a, 10))", + "row_count": 6 + } + ] + } + ] + }, + { + "Name": "TestTraceDebugSelectivity", + "Cases": [ + { + "ResultForV1": [ + { + "github.com/pingcap/tidb/pkg/planner/cardinality.Selectivity": [ + { + "Input Expressions": [ + "eq(test.t.a, 100)", + "eq(test.t.b, 350)" + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByColumnRanges": [ + { + "ID": 1, + "Ranges": [ + "[100,100]" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.ColumnStatsIsInvalid": { + "EssentialLoaded": true, + "InValidForCollPseudo": false, + "IsInvalid": false, + "NDV": 2001, + "TotalCount": 3080 + } + }, + { + "Histogram NotNull Count": 3080, + "Increase Factor": 1, + "TopN total count": 1095 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetColumnRowCount": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A4AAAAAAAABk", + "LowValueEncoded": "A4AAAAAAAABk", + "Range": "[100,100]" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAABk", + "Value": "KindInt64 100" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": 13 + }, + { + "Found": true, + "Result": 50 + } + ] + }, + { + "Error": null, + "Result": 50 + } + ] + }, + { + "End estimate range": { + "RowCount": 50, + "Type": "Point" + } + } + ] + }, + { + "Name": "a", + "Result": 50 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByColumnRanges": [ + { + "ID": 2, + "Ranges": [ + "[350,350]" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.ColumnStatsIsInvalid": { + "EssentialLoaded": true, + "InValidForCollPseudo": false, + "IsInvalid": false, + "NDV": 2001, + "TotalCount": 3080 + } + }, + { + "Histogram NotNull Count": 3080, + "Increase Factor": 1, + "TopN total count": 1098 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetColumnRowCount": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A4AAAAAAAAFe", + "LowValueEncoded": "A4AAAAAAAAFe", + "Range": "[350,350]" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAFe", + "Value": "KindInt64 350" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": -1 + }, + { + "Found": false, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 3, + 3, + 2, + 1, + 1 + ], + "Result": 1, + "Use default value": false, + "Values": [ + 2, + 2, + 3, + 4, + 4 + ] + } + }, + { + "Error": null, + "Result": 1 + } + ] + }, + { + "End estimate range": { + "RowCount": 1, + "Type": "Point" + } + } + ] + }, + { + "Name": "b", + "Result": 1 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByIndexRanges": [ + { + "ID": 1, + "Ranges": [ + "[100 350,100 350]" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.IndexStatsIsInvalid": { + "CollPseudo": false, + "IsInvalid": false, + "TotalCount": 3080 + } + }, + { + "Histogram NotNull Count": 3080, + "Increase Factor": 1, + "TopN total count": 0 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.getIndexRowCountForStatsV1": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": null, + "LowValueEncoded": null, + "Range": "[100 350,100 350]" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.getEqualCondSelectivity": [ + { + "github.com/pingcap/tidb/pkg/planner/cardinality.crossValidationSelectivity": [ + { + "github.com/pingcap/tidb/pkg/statistics.ColumnStatsIsInvalid": { + "EssentialLoaded": true, + "InValidForCollPseudo": false, + "IsInvalid": false, + "NDV": 2001, + "TotalCount": 3080 + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetColumnRowCount": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A4AAAAAAAABk", + "LowValueEncoded": "A4AAAAAAAABk", + "Range": "[100,100]" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAABk", + "Value": "KindInt64 100" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": 13 + }, + { + "Found": true, + "Result": 50 + } + ] + }, + { + "Error": null, + "Result": 50 + } + ] + }, + { + "End estimate range": { + "RowCount": 50, + "Type": "Point" + } + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.ColumnStatsIsInvalid": { + "EssentialLoaded": true, + "InValidForCollPseudo": false, + "IsInvalid": false, + "NDV": 2001, + "TotalCount": 3080 + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetColumnRowCount": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A4AAAAAAAAFe", + "LowValueEncoded": "A4AAAAAAAAFe", + "Range": "[350,350]" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAFe", + "Value": "KindInt64 350" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": -1 + }, + { + "Found": false, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 3, + 3, + 2, + 1, + 1 + ], + "Result": 1, + "Use default value": false, + "Values": [ + 2, + 2, + 3, + 4, + 4 + ] + } + }, + { + "Error": null, + "Result": 1 + } + ] + }, + { + "End estimate range": { + "RowCount": 1, + "Type": "Point" + } + } + ] + }, + { + "Index Name": "iab", + "crossValidationSelectivity": 0.0000052707033226513745, + "error": null, + "minRowCount": 1 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Index).QueryBytes": [ + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 3, + 3, + 0, + 2, + 2 + ], + "Result": 3, + "Use default value": true, + "Values": [ + 0, + 1, + 1, + 1, + 1 + ] + } + }, + { + "Result": 3 + } + ] + }, + { + "Encoded": "A4AAAAAAAABkA4AAAAAAAAFe", + "Index Name": "iab", + "Range": "[100 350,100 350]", + "Result": 0.0000052707033226513745, + "UsedColLen": 2, + "error": null + } + ] + }, + { + "End estimate range": { + "RowCount": 0.016233766233766232, + "Type": "Range" + } + } + ] + }, + { + "Name": "iab", + "Result": 0.016233766233766232 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByIndexRanges": [ + { + "ID": 2, + "Ranges": [ + "[350,350]" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.IndexStatsIsInvalid": { + "CollPseudo": false, + "IsInvalid": false, + "TotalCount": 3080 + } + }, + { + "Histogram NotNull Count": 3080, + "Increase Factor": 1, + "TopN total count": 0 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.getIndexRowCountForStatsV1": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": null, + "LowValueEncoded": null, + "Range": "[350,350]" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.getEqualCondSelectivity": [ + { + "github.com/pingcap/tidb/pkg/planner/cardinality.crossValidationSelectivity": [ + { + "github.com/pingcap/tidb/pkg/statistics.ColumnStatsIsInvalid": { + "EssentialLoaded": true, + "InValidForCollPseudo": false, + "IsInvalid": false, + "NDV": 2001, + "TotalCount": 3080 + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetColumnRowCount": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A4AAAAAAAAFe", + "LowValueEncoded": "A4AAAAAAAAFe", + "Range": "[350,350]" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAFe", + "Value": "KindInt64 350" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": -1 + }, + { + "Found": false, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 3, + 3, + 2, + 1, + 1 + ], + "Result": 1, + "Use default value": false, + "Values": [ + 2, + 2, + 3, + 4, + 4 + ] + } + }, + { + "Error": null, + "Result": 1 + } + ] + }, + { + "End estimate range": { + "RowCount": 1, + "Type": "Point" + } + } + ] + }, + { + "Index Name": "ib", + "crossValidationSelectivity": 0.0003246753246753247, + "error": null, + "minRowCount": 1 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Index).QueryBytes": [ + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 2, + 2, + 1, + 1, + 1 + ], + "Result": 1, + "Use default value": true, + "Values": [ + 1, + 1, + 1, + 2, + 2 + ] + } + }, + { + "Result": 1 + } + ] + }, + { + "Encoded": "A4AAAAAAAAFe", + "Index Name": "ib", + "Range": "[350,350]", + "Result": 0.0003246753246753247, + "UsedColLen": 1, + "error": null + } + ] + }, + { + "End estimate range": { + "RowCount": 1, + "Type": "Range" + } + } + ] + }, + { + "Name": "ib", + "Result": 1 + } + ] + }, + { + "Expressions": [ + "eq(test.t.a, 100)", + "eq(test.t.b, 350)" + ], + "Selectivity": 0.0000052707033226513745, + "partial cover": false + }, + { + "Result": 0.0000052707033226513745 + } + ] + } + ], + "ResultForV2": [ + { + "github.com/pingcap/tidb/pkg/planner/cardinality.Selectivity": [ + { + "Input Expressions": [ + "eq(test.t.a, 100)", + "eq(test.t.b, 350)" + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByColumnRanges": [ + { + "ID": 1, + "Ranges": [ + "[100,100]" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.ColumnStatsIsInvalid": { + "EssentialLoaded": true, + "InValidForCollPseudo": false, + "IsInvalid": false, + "NDV": 2000, + "TotalCount": 2980 + } + }, + { + "Histogram NotNull Count": 1980, + "Increase Factor": 1.0335570469798658, + "TopN total count": 1000 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetColumnRowCount": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A4AAAAAAAABk", + "LowValueEncoded": "A4AAAAAAAABk", + "Range": "[100,100]" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAABk", + "Value": "KindInt64 100" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": 11 + }, + { + "Found": true, + "Result": 50 + } + ] + }, + { + "Error": null, + "Result": 50 + } + ] + }, + { + "End estimate range": { + "RowCount": 51.67785234899329, + "Type": "Point" + } + } + ] + }, + { + "Name": "a", + "Result": 51.67785234899329 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByColumnRanges": [ + { + "ID": 2, + "Ranges": [ + "[350,350]" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.ColumnStatsIsInvalid": { + "EssentialLoaded": true, + "InValidForCollPseudo": false, + "IsInvalid": false, + "NDV": 2000, + "TotalCount": 2980 + } + }, + { + "Histogram NotNull Count": 1980, + "Increase Factor": 1.0335570469798658, + "TopN total count": 1000 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetColumnRowCount": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A4AAAAAAAAFe", + "LowValueEncoded": "A4AAAAAAAAFe", + "Range": "[350,350]" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAFe", + "Value": "KindInt64 350" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": -1 + }, + { + "Found": false, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).EqualRowCount": [ + { + "Locate value in buckets": { + "BucketIdx": 105, + "Exceed": false, + "InBucket": true, + "MatchLastValue": false, + "Value": "KindInt64 350" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": 848, + "Index": 105, + "Repeat": 1 + } + ] + }, + { + "Count": 0.99, + "Matched": false + } + ] + }, + { + "Error": null, + "Result": 1 + } + ] + }, + { + "End estimate range": { + "RowCount": 1.0335570469798658, + "Type": "Point" + } + } + ] + }, + { + "Name": "b", + "Result": 1.0335570469798658 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByIndexRanges": [ + { + "ID": 1, + "Ranges": [ + "[100 350,100 350]" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.IndexStatsIsInvalid": { + "CollPseudo": false, + "IsInvalid": false, + "TotalCount": 2980 + } + }, + { + "Histogram NotNull Count": 1980, + "Increase Factor": 1.0335570469798658, + "TopN total count": 1000 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.getIndexRowCountForStatsV2": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A4AAAAAAAABkA4AAAAAAAAFe", + "LowValueEncoded": "A4AAAAAAAABkA4AAAAAAAAFe", + "Range": "[100 350,100 350]" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnIndex": [ + { + "Encoded Value": "A4AAAAAAAABkA4AAAAAAAAFe" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": -1 + }, + { + "Found": false, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).EqualRowCount": [ + { + "Locate value in buckets": { + "BucketIdx": 136, + "Exceed": false, + "InBucket": true, + "MatchLastValue": false, + "Value": "KindBytes \\x03\\x80\\x00\\x00\\x00\\x00\\x00\\x00d\\x03\\x80\\x00\\x00\\x00\\x00\\x00\\x01^" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": 1096, + "Index": 136, + "Repeat": 1 + } + ] + }, + { + "Count": 0.99, + "Matched": false + } + ] + }, + { + "Result": 1 + } + ] + }, + { + "End estimate range": { + "RowCount": 1.0335570469798658, + "Type": "Point" + } + } + ] + }, + { + "Name": "iab", + "Result": 1.0335570469798658 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByIndexRanges": [ + { + "ID": 2, + "Ranges": [ + "[350,350]" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.IndexStatsIsInvalid": { + "CollPseudo": false, + "IsInvalid": false, + "TotalCount": 2980 + } + }, + { + "Histogram NotNull Count": 1980, + "Increase Factor": 1.0335570469798658, + "TopN total count": 1000 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.getIndexRowCountForStatsV2": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A4AAAAAAAAFe", + "LowValueEncoded": "A4AAAAAAAAFe", + "Range": "[350,350]" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnIndex": [ + { + "Encoded Value": "A4AAAAAAAAFe" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": -1 + }, + { + "Found": false, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).EqualRowCount": [ + { + "Locate value in buckets": { + "BucketIdx": 105, + "Exceed": false, + "InBucket": true, + "MatchLastValue": false, + "Value": "KindBytes \\x03\\x80\\x00\\x00\\x00\\x00\\x00\\x01^" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": 848, + "Index": 105, + "Repeat": 1 + } + ] + }, + { + "Count": 0.99, + "Matched": false + } + ] + }, + { + "Result": 1 + } + ] + }, + { + "End estimate range": { + "RowCount": 1.0335570469798658, + "Type": "Point" + } + } + ] + }, + { + "Name": "ib", + "Result": 1.0335570469798658 + } + ] + }, + { + "Expressions": [ + "eq(test.t.a, 100)", + "eq(test.t.b, 350)" + ], + "Selectivity": 0.00033557046979865775, + "partial cover": false + }, + { + "Result": 0.00033557046979865775 + } + ] + } + ] + }, + { + "ResultForV1": [ + { + "github.com/pingcap/tidb/pkg/planner/cardinality.Selectivity": [ + { + "Input Expressions": [ + "lt(test.t.a, -1500)", + "gt(test.t.b, 400)", + "lt(test.t.b, 403)" + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByColumnRanges": [ + { + "ID": 1, + "Ranges": [ + "[-inf,-1500)" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.ColumnStatsIsInvalid": { + "EssentialLoaded": true, + "InValidForCollPseudo": false, + "IsInvalid": false, + "NDV": 2001, + "TotalCount": 3080 + } + }, + { + "Histogram NotNull Count": 3080, + "Increase Factor": 1, + "TopN total count": 1095 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetColumnRowCount": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A3////////ok", + "LowValueEncoded": "AQ==", + "Range": "[-inf,-1500)" + } + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).LessRowCountWithBktIdx": [ + { + "Locate value in buckets": { + "BucketIdx": 0, + "Exceed": false, + "InBucket": false, + "MatchLastValue": false, + "Value": "KindMinNotNull " + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": -1, + "Index": -1, + "Repeat": -1 + }, + { + "Count": 50, + "Index": 0, + "Repeat": 50 + } + ] + }, + { + "Bucket idx": 0, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).LessRowCountWithBktIdx": [ + { + "Locate value in buckets": { + "BucketIdx": 0, + "Exceed": false, + "InBucket": false, + "MatchLastValue": false, + "Value": "KindInt64 -1500" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": -1, + "Index": -1, + "Repeat": -1 + }, + { + "Count": 50, + "Index": 0, + "Repeat": 50 + } + ] + }, + { + "Bucket idx": 0, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).EqualRowCount": [ + { + "Locate value in buckets": { + "BucketIdx": 0, + "Exceed": false, + "InBucket": false, + "MatchLastValue": false, + "Value": "KindMinNotNull " + } + }, + { + "Count": 0, + "Matched": false + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).OutOfRangeRowCount": [ + { + "lDatum": "KindMinNotNull ", + "modifyCount": 0, + "rDatum": "KindInt64 -1500" + }, + { + "commonPrefix": 0, + "lScalar": -1.7976931348623157e+308, + "rScalar": -1500, + "unsigned": false + }, + { + "boundL": -7000, + "boundR": 11000, + "histL": -1000, + "histR": 5000, + "lPercent": 0.8402777777777778, + "rPercent": 0, + "rowCount": 1294.0277777777778 + }, + { + "Result": 0 + } + ] + }, + { + "End estimate range": { + "RowCount": 0, + "Type": "Range" + } + } + ] + }, + { + "Name": "a", + "Result": 1 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByColumnRanges": [ + { + "ID": 2, + "Ranges": [ + "(400,403)" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.ColumnStatsIsInvalid": { + "EssentialLoaded": true, + "InValidForCollPseudo": false, + "IsInvalid": false, + "NDV": 2001, + "TotalCount": 3080 + } + }, + { + "Histogram NotNull Count": 3080, + "Increase Factor": 1, + "TopN total count": 1098 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetColumnRowCount": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A4AAAAAAAAGT", + "LowValueEncoded": "A4AAAAAAAAGQ", + "Range": "(400,403)" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAGQ", + "Value": "KindInt64 401" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": -1 + }, + { + "Found": false, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 1, + 2, + 2, + 1, + 1 + ], + "Result": 1, + "Use default value": false, + "Values": [ + 2, + 2, + 2, + 3, + 3 + ] + } + }, + { + "Error": null, + "Result": 1 + } + ] + }, + { + "End estimate range": { + "RowCount": 1, + "Type": "Small range in ver1 stats" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAGQ", + "Value": "KindInt64 402" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": -1 + }, + { + "Found": false, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 1, + 3, + 2, + 2, + 2 + ], + "Result": 1, + "Use default value": false, + "Values": [ + 2, + 3, + 3, + 3, + 4 + ] + } + }, + { + "Error": null, + "Result": 1 + } + ] + }, + { + "End estimate range": { + "RowCount": 1, + "Type": "Small range in ver1 stats" + } + } + ] + }, + { + "Name": "b", + "Result": 2 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByIndexRanges": [ + { + "ID": 1, + "Ranges": [ + "[-inf,-1500)" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.IndexStatsIsInvalid": { + "CollPseudo": false, + "IsInvalid": false, + "TotalCount": 3080 + } + }, + { + "Histogram NotNull Count": 3080, + "Increase Factor": 1, + "TopN total count": 0 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.getIndexRowCountForStatsV1": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": null, + "LowValueEncoded": null, + "Range": "[-inf,-1500)" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.getIndexRowCountForStatsV2": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A3////////ok", + "LowValueEncoded": "AQ==", + "Range": "[-inf,-1500)" + } + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).LessRowCountWithBktIdx": [ + { + "Locate value in buckets": { + "BucketIdx": 0, + "Exceed": false, + "InBucket": false, + "MatchLastValue": false, + "Value": "KindBytes \\x01" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": -1, + "Index": -1, + "Repeat": -1 + }, + { + "Count": 57, + "Index": 0, + "Repeat": 1 + } + ] + }, + { + "Bucket idx": 0, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).LessRowCountWithBktIdx": [ + { + "Locate value in buckets": { + "BucketIdx": 0, + "Exceed": false, + "InBucket": false, + "MatchLastValue": false, + "Value": "KindBytes \\x03\\x7f\\xff\\xff\\xff\\xff\\xff\\xfa$" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": -1, + "Index": -1, + "Repeat": -1 + }, + { + "Count": 57, + "Index": 0, + "Repeat": 1 + } + ] + }, + { + "Bucket idx": 0, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).EqualRowCount": [ + { + "Locate value in buckets": { + "BucketIdx": 0, + "Exceed": false, + "InBucket": false, + "MatchLastValue": false, + "Value": "KindBytes \\x01" + } + }, + { + "Count": 0, + "Matched": false + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).OutOfRangeRowCount": [ + { + "lDatum": "KindBytes \\x01", + "modifyCount": 0, + "rDatum": "KindBytes \\x03\\x7f\\xff\\xff\\xff\\xff\\xff\\xfa$" + }, + { + "commonPrefix": 0, + "lScalar": 72057594037927940, + "rScalar": 252201579132747780, + "unsigned": false + }, + { + "boundL": 252201579132747740, + "boundR": 252201579132747840, + "histL": 252201579132747780, + "histR": 252201579132747800, + "lPercent": 1, + "rPercent": 0, + "rowCount": 1540 + }, + { + "Result": 0 + } + ] + }, + { + "End estimate range": { + "RowCount": 0, + "Type": "Range" + } + } + ] + }, + { + "End estimate range": { + "RowCount": 1, + "Type": "Range" + } + } + ] + }, + { + "Name": "iab", + "Result": 1 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByIndexRanges": [ + { + "ID": 2, + "Ranges": [ + "(400,403)" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.IndexStatsIsInvalid": { + "CollPseudo": false, + "IsInvalid": false, + "TotalCount": 3080 + } + }, + { + "Histogram NotNull Count": 3080, + "Increase Factor": 1, + "TopN total count": 0 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.getIndexRowCountForStatsV1": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": null, + "LowValueEncoded": null, + "Range": "(400,403)" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.getEqualCondSelectivity": [ + { + "github.com/pingcap/tidb/pkg/planner/cardinality.crossValidationSelectivity": [ + { + "github.com/pingcap/tidb/pkg/statistics.ColumnStatsIsInvalid": { + "EssentialLoaded": true, + "InValidForCollPseudo": false, + "IsInvalid": false, + "NDV": 2001, + "TotalCount": 3080 + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetColumnRowCount": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A4AAAAAAAAGT", + "LowValueEncoded": "A4AAAAAAAAGQ", + "Range": "[400,403]" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAGQ", + "Value": "KindInt64 400" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": 7 + }, + { + "Found": true, + "Result": 50 + } + ] + }, + { + "Error": null, + "Result": 50 + } + ] + }, + { + "End estimate range": { + "RowCount": 50, + "Type": "Small range in ver1 stats" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAGQ", + "Value": "KindInt64 401" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": -1 + }, + { + "Found": false, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 1, + 2, + 2, + 1, + 1 + ], + "Result": 1, + "Use default value": false, + "Values": [ + 2, + 2, + 2, + 3, + 3 + ] + } + }, + { + "Error": null, + "Result": 1 + } + ] + }, + { + "End estimate range": { + "RowCount": 1, + "Type": "Small range in ver1 stats" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAGQ", + "Value": "KindInt64 402" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": -1 + }, + { + "Found": false, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 1, + 3, + 2, + 2, + 2 + ], + "Result": 1, + "Use default value": false, + "Values": [ + 2, + 3, + 3, + 3, + 4 + ] + } + }, + { + "Error": null, + "Result": 1 + } + ] + }, + { + "End estimate range": { + "RowCount": 1, + "Type": "Small range in ver1 stats" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAGQ", + "Value": "KindInt64 403" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": -1 + }, + { + "Found": false, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 2, + 2, + 1, + 2, + 1 + ], + "Result": 1, + "Use default value": false, + "Values": [ + 2, + 2, + 3, + 3, + 3 + ] + } + }, + { + "Error": null, + "Result": 1 + } + ] + }, + { + "End estimate range": { + "RowCount": 1, + "Type": "Small range in ver1 stats" + } + } + ] + }, + { + "Index Name": "ib", + "crossValidationSelectivity": 0.017207792207792207, + "error": null, + "minRowCount": 53 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Index).QueryBytes": [ + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 1, + 2, + 2, + 1, + 3 + ], + "Result": 1, + "Use default value": false, + "Values": [ + 1, + 1, + 2, + 2, + 3 + ] + } + }, + { + "Result": 1 + } + ] + }, + { + "Encoded": "A4AAAAAAAAGS", + "Index Name": "ib", + "Range": "(400,403)", + "Result": 0.0003246753246753247, + "UsedColLen": 1, + "error": null + } + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.getEqualCondSelectivity": [ + { + "github.com/pingcap/tidb/pkg/planner/cardinality.crossValidationSelectivity": [ + { + "github.com/pingcap/tidb/pkg/statistics.ColumnStatsIsInvalid": { + "EssentialLoaded": true, + "InValidForCollPseudo": false, + "IsInvalid": false, + "NDV": 2001, + "TotalCount": 3080 + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetColumnRowCount": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A4AAAAAAAAGT", + "LowValueEncoded": "A4AAAAAAAAGQ", + "Range": "[400,403]" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAGQ", + "Value": "KindInt64 400" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": 7 + }, + { + "Found": true, + "Result": 50 + } + ] + }, + { + "Error": null, + "Result": 50 + } + ] + }, + { + "End estimate range": { + "RowCount": 50, + "Type": "Small range in ver1 stats" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAGQ", + "Value": "KindInt64 401" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": -1 + }, + { + "Found": false, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 1, + 2, + 2, + 1, + 1 + ], + "Result": 1, + "Use default value": false, + "Values": [ + 2, + 2, + 2, + 3, + 3 + ] + } + }, + { + "Error": null, + "Result": 1 + } + ] + }, + { + "End estimate range": { + "RowCount": 1, + "Type": "Small range in ver1 stats" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAGQ", + "Value": "KindInt64 402" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": -1 + }, + { + "Found": false, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 1, + 3, + 2, + 2, + 2 + ], + "Result": 1, + "Use default value": false, + "Values": [ + 2, + 3, + 3, + 3, + 4 + ] + } + }, + { + "Error": null, + "Result": 1 + } + ] + }, + { + "End estimate range": { + "RowCount": 1, + "Type": "Small range in ver1 stats" + } + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAGQ", + "Value": "KindInt64 403" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": -1 + }, + { + "Found": false, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 2, + 2, + 1, + 2, + 1 + ], + "Result": 1, + "Use default value": false, + "Values": [ + 2, + 2, + 3, + 3, + 3 + ] + } + }, + { + "Error": null, + "Result": 1 + } + ] + }, + { + "End estimate range": { + "RowCount": 1, + "Type": "Small range in ver1 stats" + } + } + ] + }, + { + "Index Name": "ib", + "crossValidationSelectivity": 0.017207792207792207, + "error": null, + "minRowCount": 53 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Index).QueryBytes": [ + { + "github.com/pingcap/tidb/pkg/statistics.(*CMSketch).queryHashValue": { + "Origin Values": [ + 2, + 3, + 1, + 3, + 1 + ], + "Result": 1, + "Use default value": false, + "Values": [ + 1, + 1, + 2, + 3, + 3 + ] + } + }, + { + "Result": 1 + } + ] + }, + { + "Encoded": "A4AAAAAAAAGS", + "Index Name": "ib", + "Range": "(400,403)", + "Result": 0.0003246753246753247, + "UsedColLen": 1, + "error": null + } + ] + }, + { + "End estimate range": { + "RowCount": 2, + "Type": "Range" + } + } + ] + }, + { + "Name": "ib", + "Result": 2 + } + ] + }, + { + "Expressions": [ + "gt(test.t.b, 400)", + "lt(test.t.b, 403)" + ], + "Selectivity": 0.0006493506493506494, + "partial cover": false + }, + { + "Expressions": [ + "lt(test.t.a, -1500)" + ], + "Selectivity": 0.0003246753246753247, + "partial cover": false + }, + { + "Result": 2.1082813290605499e-7 + } + ] + } + ], + "ResultForV2": [ + { + "github.com/pingcap/tidb/pkg/planner/cardinality.Selectivity": [ + { + "Input Expressions": [ + "lt(test.t.a, -1500)", + "gt(test.t.b, 400)", + "lt(test.t.b, 403)" + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByColumnRanges": [ + { + "ID": 1, + "Ranges": [ + "[-inf,-1500)" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.ColumnStatsIsInvalid": { + "EssentialLoaded": true, + "InValidForCollPseudo": false, + "IsInvalid": false, + "NDV": 2000, + "TotalCount": 2980 + } + }, + { + "Histogram NotNull Count": 1980, + "Increase Factor": 1.0335570469798658, + "TopN total count": 1000 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetColumnRowCount": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A3////////ok", + "LowValueEncoded": "AQ==", + "Range": "[-inf,-1500)" + } + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).LessRowCountWithBktIdx": [ + { + "Locate value in buckets": { + "BucketIdx": 0, + "Exceed": false, + "InBucket": false, + "MatchLastValue": false, + "Value": "KindMinNotNull " + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": -1, + "Index": -1, + "Repeat": -1 + }, + { + "Count": 8, + "Index": 0, + "Repeat": 1 + } + ] + }, + { + "Bucket idx": 0, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).LessRowCountWithBktIdx": [ + { + "Locate value in buckets": { + "BucketIdx": 0, + "Exceed": false, + "InBucket": false, + "MatchLastValue": false, + "Value": "KindInt64 -1500" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": -1, + "Index": -1, + "Repeat": -1 + }, + { + "Count": 8, + "Index": 0, + "Repeat": 1 + } + ] + }, + { + "Bucket idx": 0, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).EqualRowCount": [ + { + "Locate value in buckets": { + "BucketIdx": 0, + "Exceed": false, + "InBucket": false, + "MatchLastValue": false, + "Value": "KindMinNotNull " + } + }, + { + "Count": 0, + "Matched": false + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).BetweenCount": { + "Result": 0 + } + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).OutOfRangeRowCount": [ + { + "lDatum": "KindMinNotNull ", + "modifyCount": 100, + "rDatum": "KindInt64 -1500" + }, + { + "commonPrefix": 0, + "lScalar": -1.7976931348623157e+308, + "rScalar": -1500, + "unsigned": false + }, + { + "boundL": -2997, + "boundR": 2997, + "histL": -999, + "histR": 999, + "lPercent": 0.5613744375005636, + "rPercent": 0, + "rowCount": 555.760693125558 + }, + { + "Result": 100 + } + ] + }, + { + "End estimate range": { + "RowCount": 100, + "Type": "Range" + } + } + ] + }, + { + "Name": "a", + "Result": 100 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByColumnRanges": [ + { + "ID": 2, + "Ranges": [ + "(400,403)" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.ColumnStatsIsInvalid": { + "EssentialLoaded": true, + "InValidForCollPseudo": false, + "IsInvalid": false, + "NDV": 2000, + "TotalCount": 2980 + } + }, + { + "Histogram NotNull Count": 1980, + "Increase Factor": 1.0335570469798658, + "TopN total count": 1000 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetColumnRowCount": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A4AAAAAAAAGT", + "LowValueEncoded": "A4AAAAAAAAGQ", + "Range": "(400,403)" + } + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).LessRowCountWithBktIdx": [ + { + "Locate value in buckets": { + "BucketIdx": 111, + "Exceed": false, + "InBucket": true, + "MatchLastValue": false, + "Value": "KindInt64 400" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": 888, + "Index": 110, + "Repeat": 1 + }, + { + "Count": 896, + "Index": 111, + "Repeat": 1 + } + ] + }, + { + "Bucket idx": 111, + "Result": 890.625 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).LessRowCountWithBktIdx": [ + { + "Locate value in buckets": { + "BucketIdx": 111, + "Exceed": false, + "InBucket": true, + "MatchLastValue": false, + "Value": "KindInt64 403" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": 888, + "Index": 110, + "Repeat": 1 + }, + { + "Count": 896, + "Index": 111, + "Repeat": 1 + } + ] + }, + { + "Bucket idx": 111, + "Result": 893.25 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).EqualRowCount": [ + { + "Locate value in buckets": { + "BucketIdx": 111, + "Exceed": false, + "InBucket": true, + "MatchLastValue": false, + "Value": "KindInt64 400" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": 896, + "Index": 111, + "Repeat": 1 + } + ] + }, + { + "Count": 0.99, + "Matched": false + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).BetweenCount": [ + { + "Related TopN Range": { + "Count": [ + 50 + ], + "FirstEncoded": "A4AAAAAAAAGQ", + "FirstIdx": 9, + "LastEncoded": "A4AAAAAAAAGQ", + "LastIdx": 9 + } + }, + { + "Result": 50 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.equalRowCountOnColumn": [ + { + "Encoded": "A4AAAAAAAAGQ", + "Value": "KindInt64 400" + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).QueryTopN": [ + { + "FindTopN idx": 9 + }, + { + "Found": true, + "Result": 50 + } + ] + }, + { + "Error": null, + "Result": 50 + } + ] + }, + { + "End estimate range": { + "RowCount": 2.713087248322148, + "Type": "Range" + } + } + ] + }, + { + "Name": "b", + "Result": 2.713087248322148 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByIndexRanges": [ + { + "ID": 1, + "Ranges": [ + "[-inf,-1500)" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.IndexStatsIsInvalid": { + "CollPseudo": false, + "IsInvalid": false, + "TotalCount": 2980 + } + }, + { + "Histogram NotNull Count": 1980, + "Increase Factor": 1.0335570469798658, + "TopN total count": 1000 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.getIndexRowCountForStatsV2": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A3////////ok", + "LowValueEncoded": "AQ==", + "Range": "[-inf,-1500)" + } + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).LessRowCountWithBktIdx": [ + { + "Locate value in buckets": { + "BucketIdx": 0, + "Exceed": false, + "InBucket": false, + "MatchLastValue": false, + "Value": "KindBytes \\x01" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": -1, + "Index": -1, + "Repeat": -1 + }, + { + "Count": 8, + "Index": 0, + "Repeat": 1 + } + ] + }, + { + "Bucket idx": 0, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).LessRowCountWithBktIdx": [ + { + "Locate value in buckets": { + "BucketIdx": 0, + "Exceed": false, + "InBucket": false, + "MatchLastValue": false, + "Value": "KindBytes \\x03\\x7f\\xff\\xff\\xff\\xff\\xff\\xfa$" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": -1, + "Index": -1, + "Repeat": -1 + }, + { + "Count": 8, + "Index": 0, + "Repeat": 1 + } + ] + }, + { + "Bucket idx": 0, + "Result": 0 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).EqualRowCount": [ + { + "Locate value in buckets": { + "BucketIdx": 0, + "Exceed": false, + "InBucket": false, + "MatchLastValue": false, + "Value": "KindBytes \\x01" + } + }, + { + "Count": 0, + "Matched": false + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).BetweenCount": { + "Result": 0 + } + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).OutOfRangeRowCount": [ + { + "lDatum": "KindBytes \\x01", + "modifyCount": 100, + "rDatum": "KindBytes \\x03\\x7f\\xff\\xff\\xff\\xff\\xff\\xfa$" + }, + { + "commonPrefix": 0, + "lScalar": 72057594037927940, + "rScalar": 252201579132747780, + "unsigned": false + }, + { + "Result": 0 + } + ] + }, + { + "End estimate range": { + "RowCount": 0, + "Type": "Range" + } + } + ] + }, + { + "Name": "iab", + "Result": 1 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.GetRowCountByIndexRanges": [ + { + "ID": 2, + "Ranges": [ + "(400,403)" + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.IndexStatsIsInvalid": { + "CollPseudo": false, + "IsInvalid": false, + "TotalCount": 2980 + } + }, + { + "Histogram NotNull Count": 1980, + "Increase Factor": 1.0335570469798658, + "TopN total count": 1000 + }, + { + "github.com/pingcap/tidb/pkg/planner/cardinality.getIndexRowCountForStatsV2": [ + { + "Start estimate range": { + "CurrentRowCount": 0, + "HighValueEncoded": "A4AAAAAAAAGT", + "LowValueEncoded": "A4AAAAAAAAGQ", + "Range": "(400,403)" + } + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).LessRowCountWithBktIdx": [ + { + "Locate value in buckets": { + "BucketIdx": 111, + "Exceed": false, + "InBucket": true, + "MatchLastValue": false, + "Value": "KindBytes \\x03\\x80\\x00\\x00\\x00\\x00\\x00\\x01\\x91" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": 888, + "Index": 110, + "Repeat": 1 + }, + { + "Count": 896, + "Index": 111, + "Repeat": 1 + } + ] + }, + { + "Bucket idx": 111, + "Result": 891.5 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).LessRowCountWithBktIdx": [ + { + "Locate value in buckets": { + "BucketIdx": 111, + "Exceed": false, + "InBucket": true, + "MatchLastValue": false, + "Value": "KindBytes \\x03\\x80\\x00\\x00\\x00\\x00\\x00\\x01\\x93" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": 888, + "Index": 110, + "Repeat": 1 + }, + { + "Count": 896, + "Index": 111, + "Repeat": 1 + } + ] + }, + { + "Bucket idx": 111, + "Result": 893.25 + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*Histogram).EqualRowCount": [ + { + "Locate value in buckets": { + "BucketIdx": 111, + "Exceed": false, + "InBucket": true, + "MatchLastValue": false, + "Value": "KindBytes \\x03\\x80\\x00\\x00\\x00\\x00\\x00\\x01\\x91" + } + }, + { + "Related Buckets in Histogram": [ + { + "Count": 896, + "Index": 111, + "Repeat": 1 + } + ] + }, + { + "Count": 0.99, + "Matched": false + } + ] + }, + { + "github.com/pingcap/tidb/pkg/statistics.(*TopN).BetweenCount": { + "Result": 0 + } + }, + { + "End estimate range": { + "RowCount": 1.8087248322147653, + "Type": "Range" + } + } + ] + }, + { + "Name": "ib", + "Result": 1.8087248322147653 + } + ] + }, + { + "Expressions": [ + "gt(test.t.b, 400)", + "lt(test.t.b, 403)" + ], + "Selectivity": 0.0005872483221476511, + "partial cover": false + }, + { + "Expressions": [ + "lt(test.t.a, -1500)" + ], + "Selectivity": 0.0003246753246753247, + "partial cover": false + }, + { + "Result": 1.9066503965832828e-7 + } + ] + } + ] + } + ] + }, + { + "Name": "TestIndexJoinInnerRowCountUpperBound", + "Cases": [ + { + "Query": "explain format = 'brief' select /*+ inl_join(t2) */ * from (select * from t where t.a < 1) as t1 join t t2 where t2.a = 0 and t1.a = t2.b", + "Result": [ + "Projection 2000.00 root test.t.a, test.t.b, test.t.a, test.t.b", + "└─IndexJoin 2000.00 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.t.b, equal cond:eq(test.t.a, test.t.b)", + " ├─TableReader(Build) 1000.00 root data:Selection", + " │ └─Selection 1000.00 cop[tikv] lt(test.t.a, 1), not(isnull(test.t.a))", + " │ └─TableFullScan 500000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─IndexLookUp(Probe) 2000.00 root ", + " ├─Selection(Build) 1000000.00 cop[tikv] lt(test.t.b, 1), not(isnull(test.t.b))", + " │ └─IndexRangeScan 500000000.00 cop[tikv] table:t2, index:idx(b) range: decided by [eq(test.t.b, test.t.a)], keep order:false, stats:pseudo", + " └─Selection(Probe) 2000.00 cop[tikv] eq(test.t.a, 0)", + " └─TableRowIDScan 1000000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ] + }, + { + "Query": "set @@tidb_opt_fix_control = '44855:ON'", + "Result": null + }, + { + "Query": "explain format = 'brief' select /*+ inl_join(t2) */ * from (select * from t where t.a < 1) as t1 join t t2 where t2.a = 0 and t1.a = t2.b", + "Result": [ + "Projection 2000.00 root test.t.a, test.t.b, test.t.a, test.t.b", + "└─IndexJoin 2000.00 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.t.b, equal cond:eq(test.t.a, test.t.b)", + " ├─TableReader(Build) 1000.00 root data:Selection", + " │ └─Selection 1000.00 cop[tikv] lt(test.t.a, 1), not(isnull(test.t.a))", + " │ └─TableFullScan 500000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─IndexLookUp(Probe) 2000.00 root ", + " ├─Selection(Build) 1000000.00 cop[tikv] lt(test.t.b, 1), not(isnull(test.t.b))", + " │ └─IndexRangeScan 1000000.00 cop[tikv] table:t2, index:idx(b) range: decided by [eq(test.t.b, test.t.a)], keep order:false, stats:pseudo", + " └─Selection(Probe) 2000.00 cop[tikv] eq(test.t.a, 0)", + " └─TableRowIDScan 1000000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestSubsetIdxCardinality", + "Cases": [ + { + "Query": "explain format = 'brief' select distinct a from t", + "Result": [ + "StreamAgg 31.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", + "└─IndexReader 31.00 root index:StreamAgg", + " └─StreamAgg 31.00 cop[tikv] group by:test.t.a, ", + " └─IndexFullScan 1280.00 cop[tikv] table:t, index:iabc(a, b, c) keep order:true" + ] + }, + { + "Query": "explain format = 'brief' select distinct b from t", + "Result": [ + "HashAgg 14.00 root group by:test.t.b, funcs:firstrow(test.t.b)->test.t.b", + "└─IndexReader 14.00 root index:HashAgg", + " └─HashAgg 14.00 cop[tikv] group by:test.t.b, ", + " └─IndexFullScan 1280.00 cop[tikv] table:t, index:iabc(a, b, c) keep order:false" + ] + }, + { + "Query": "explain format = 'brief' select distinct a, b, c from t", + "Result": [ + "StreamAgg 70.00 root group by:test.t.a, test.t.b, test.t.c, funcs:firstrow(test.t.a)->test.t.a, funcs:firstrow(test.t.b)->test.t.b, funcs:firstrow(test.t.c)->test.t.c", + "└─IndexReader 70.00 root index:StreamAgg", + " └─StreamAgg 70.00 cop[tikv] group by:test.t.a, test.t.b, test.t.c, ", + " └─IndexFullScan 1280.00 cop[tikv] table:t, index:iabc(a, b, c) keep order:true" + ] + }, + { + "Query": "explain format = 'brief' select a, b from t where a = 1 and b = 1 and c = 1", + "Result": [ + "IndexReader 16.00 root index:Projection", + "└─Projection 16.00 cop[tikv] test.t.a, test.t.b", + " └─IndexRangeScan 16.00 cop[tikv] table:t, index:iabc(a, b, c) range:[1 1 1,1 1 1], keep order:false" + ] + }, + { + "Query": "explain format = 'brief' select a, b from t where a = 1 and b = 1", + "Result": [ + "IndexReader 20.32 root index:IndexRangeScan", + "└─IndexRangeScan 20.32 cop[tikv] table:t, index:iabc(a, b, c) range:[1 1,1 1], keep order:false" + ] + } + ] + } +] diff --git a/pkg/planner/core/casetest/partition/testdata/integration_partition_suite_out.json b/pkg/planner/core/casetest/partition/testdata/integration_partition_suite_out.json new file mode 100644 index 0000000000000..e70e65ab26ccd --- /dev/null +++ b/pkg/planner/core/casetest/partition/testdata/integration_partition_suite_out.json @@ -0,0 +1,1855 @@ +[ + { + "Name": "TestListPartitionPruning", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from tlist where a in (2)", + "DynamicPlan": [ + "TableReader 10.00 root partition:p0 data:Selection", + "└─Selection 10.00 cop[tikv] eq(list_partition_pruning.tlist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] eq(list_partition_pruning.tlist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tlist where a in (7)", + "DynamicPlan": [ + "TableReader 10.00 root partition:p2 data:Selection", + "└─Selection 10.00 cop[tikv] eq(list_partition_pruning.tlist.a, 7)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] eq(list_partition_pruning.tlist.a, 7)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tlist where a in (0,5)", + "DynamicPlan": [ + "TableReader 20.00 root partition:p0,p1 data:Selection", + "└─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 40.00 root ", + "├─TableReader 20.00 root data:Selection", + "│ └─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo", + "└─TableReader 20.00 root data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tcollist where a in (2)", + "DynamicPlan": [ + "TableReader 10.00 root partition:p0 data:Selection", + "└─Selection 10.00 cop[tikv] eq(list_partition_pruning.tcollist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] eq(list_partition_pruning.tcollist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tcollist where a in (7)", + "DynamicPlan": [ + "TableReader 10.00 root partition:p2 data:Selection", + "└─Selection 10.00 cop[tikv] eq(list_partition_pruning.tcollist.a, 7)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] eq(list_partition_pruning.tcollist.a, 7)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tcollist where a in (0,5)", + "DynamicPlan": [ + "TableReader 20.00 root partition:p0,p1 data:Selection", + "└─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 40.00 root ", + "├─TableReader 20.00 root data:Selection", + "│ └─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo", + "└─TableReader 20.00 root data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tlist where a not in (0, 1, 2, 3, 4, 5, 6, 7, 8)", + "DynamicPlan": [ + "TableReader 3583.33 root partition:all data:Selection", + "└─Selection 3583.33 cop[tikv] not(in(list_partition_pruning.tlist.a, 0, 1, 2, 3, 4, 5, 6, 7, 8))", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 14333.33 root ", + "├─TableReader 3583.33 root data:Selection", + "│ └─Selection 3583.33 cop[tikv] not(in(list_partition_pruning.tlist.a, 0, 1, 2, 3, 4, 5, 6, 7, 8))", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo", + "├─TableReader 3583.33 root data:Selection", + "│ └─Selection 3583.33 cop[tikv] not(in(list_partition_pruning.tlist.a, 0, 1, 2, 3, 4, 5, 6, 7, 8))", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p1 keep order:false, stats:pseudo", + "├─TableReader 3583.33 root data:Selection", + "│ └─Selection 3583.33 cop[tikv] not(in(list_partition_pruning.tlist.a, 0, 1, 2, 3, 4, 5, 6, 7, 8))", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p2 keep order:false, stats:pseudo", + "└─TableReader 3583.33 root data:Selection", + " └─Selection 3583.33 cop[tikv] not(in(list_partition_pruning.tlist.a, 0, 1, 2, 3, 4, 5, 6, 7, 8))", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p3 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tcollist where a not in (0, 1, 2, 3, 4, 5, 6, 7, 8)", + "DynamicPlan": [ + "TableReader 3583.33 root partition:p3 data:Selection", + "└─Selection 3583.33 cop[tikv] not(in(list_partition_pruning.tcollist.a, 0, 1, 2, 3, 4, 5, 6, 7, 8))", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 3583.33 root data:Selection", + "└─Selection 3583.33 cop[tikv] not(in(list_partition_pruning.tcollist.a, 0, 1, 2, 3, 4, 5, 6, 7, 8))", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p3 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tlist where a >= 7", + "DynamicPlan": [ + "TableReader 3333.33 root partition:all data:Selection", + "└─Selection 3333.33 cop[tikv] ge(list_partition_pruning.tlist.a, 7)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 13333.33 root ", + "├─TableReader 3333.33 root data:Selection", + "│ └─Selection 3333.33 cop[tikv] ge(list_partition_pruning.tlist.a, 7)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo", + "├─TableReader 3333.33 root data:Selection", + "│ └─Selection 3333.33 cop[tikv] ge(list_partition_pruning.tlist.a, 7)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p1 keep order:false, stats:pseudo", + "├─TableReader 3333.33 root data:Selection", + "│ └─Selection 3333.33 cop[tikv] ge(list_partition_pruning.tlist.a, 7)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p2 keep order:false, stats:pseudo", + "└─TableReader 3333.33 root data:Selection", + " └─Selection 3333.33 cop[tikv] ge(list_partition_pruning.tlist.a, 7)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p3 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tcollist where a >= 7", + "DynamicPlan": [ + "TableReader 3333.33 root partition:p2,p3 data:Selection", + "└─Selection 3333.33 cop[tikv] ge(list_partition_pruning.tcollist.a, 7)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 6666.67 root ", + "├─TableReader 3333.33 root data:Selection", + "│ └─Selection 3333.33 cop[tikv] ge(list_partition_pruning.tcollist.a, 7)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p2 keep order:false, stats:pseudo", + "└─TableReader 3333.33 root data:Selection", + " └─Selection 3333.33 cop[tikv] ge(list_partition_pruning.tcollist.a, 7)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p3 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tlist where a <= 7", + "DynamicPlan": [ + "TableReader 3323.33 root partition:all data:Selection", + "└─Selection 3323.33 cop[tikv] le(list_partition_pruning.tlist.a, 7)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 13293.33 root ", + "├─TableReader 3323.33 root data:Selection", + "│ └─Selection 3323.33 cop[tikv] le(list_partition_pruning.tlist.a, 7)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo", + "├─TableReader 3323.33 root data:Selection", + "│ └─Selection 3323.33 cop[tikv] le(list_partition_pruning.tlist.a, 7)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p1 keep order:false, stats:pseudo", + "├─TableReader 3323.33 root data:Selection", + "│ └─Selection 3323.33 cop[tikv] le(list_partition_pruning.tlist.a, 7)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p2 keep order:false, stats:pseudo", + "└─TableReader 3323.33 root data:Selection", + " └─Selection 3323.33 cop[tikv] le(list_partition_pruning.tlist.a, 7)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p3 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tcollist where a <= 7", + "DynamicPlan": [ + "TableReader 3323.33 root partition:p0,p1,p2 data:Selection", + "└─Selection 3323.33 cop[tikv] le(list_partition_pruning.tcollist.a, 7)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 9970.00 root ", + "├─TableReader 3323.33 root data:Selection", + "│ └─Selection 3323.33 cop[tikv] le(list_partition_pruning.tcollist.a, 7)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo", + "├─TableReader 3323.33 root data:Selection", + "│ └─Selection 3323.33 cop[tikv] le(list_partition_pruning.tcollist.a, 7)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p1 keep order:false, stats:pseudo", + "└─TableReader 3323.33 root data:Selection", + " └─Selection 3323.33 cop[tikv] le(list_partition_pruning.tcollist.a, 7)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tlist where a=0 or a=7", + "DynamicPlan": [ + "TableReader 20.00 root partition:p0,p2 data:Selection", + "└─Selection 20.00 cop[tikv] or(eq(list_partition_pruning.tlist.a, 0), eq(list_partition_pruning.tlist.a, 7))", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 40.00 root ", + "├─TableReader 20.00 root data:Selection", + "│ └─Selection 20.00 cop[tikv] or(eq(list_partition_pruning.tlist.a, 0), eq(list_partition_pruning.tlist.a, 7))", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo", + "└─TableReader 20.00 root data:Selection", + " └─Selection 20.00 cop[tikv] or(eq(list_partition_pruning.tlist.a, 0), eq(list_partition_pruning.tlist.a, 7))", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tlist where a=0 or a=1 or a=6 or a=11", + "DynamicPlan": [ + "TableReader 40.00 root partition:p0,p2,p3 data:Selection", + "└─Selection 40.00 cop[tikv] or(or(eq(list_partition_pruning.tlist.a, 0), eq(list_partition_pruning.tlist.a, 1)), or(eq(list_partition_pruning.tlist.a, 6), eq(list_partition_pruning.tlist.a, 11)))", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 120.00 root ", + "├─TableReader 40.00 root data:Selection", + "│ └─Selection 40.00 cop[tikv] or(or(eq(list_partition_pruning.tlist.a, 0), eq(list_partition_pruning.tlist.a, 1)), or(eq(list_partition_pruning.tlist.a, 6), eq(list_partition_pruning.tlist.a, 11)))", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo", + "├─TableReader 40.00 root data:Selection", + "│ └─Selection 40.00 cop[tikv] or(or(eq(list_partition_pruning.tlist.a, 0), eq(list_partition_pruning.tlist.a, 1)), or(eq(list_partition_pruning.tlist.a, 6), eq(list_partition_pruning.tlist.a, 11)))", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p2 keep order:false, stats:pseudo", + "└─TableReader 40.00 root data:Selection", + " └─Selection 40.00 cop[tikv] or(or(eq(list_partition_pruning.tlist.a, 0), eq(list_partition_pruning.tlist.a, 1)), or(eq(list_partition_pruning.tlist.a, 6), eq(list_partition_pruning.tlist.a, 11)))", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p3 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tcollist where a=0 or a=7", + "DynamicPlan": [ + "TableReader 20.00 root partition:p0,p2 data:Selection", + "└─Selection 20.00 cop[tikv] or(eq(list_partition_pruning.tcollist.a, 0), eq(list_partition_pruning.tcollist.a, 7))", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 40.00 root ", + "├─TableReader 20.00 root data:Selection", + "│ └─Selection 20.00 cop[tikv] or(eq(list_partition_pruning.tcollist.a, 0), eq(list_partition_pruning.tcollist.a, 7))", + "│ └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo", + "└─TableReader 20.00 root data:Selection", + " └─Selection 20.00 cop[tikv] or(eq(list_partition_pruning.tcollist.a, 0), eq(list_partition_pruning.tcollist.a, 7))", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tcollist where a=0 or a=1 or a=6 or a=11", + "DynamicPlan": [ + "TableReader 40.00 root partition:p0,p2,p3 data:Selection", + "└─Selection 40.00 cop[tikv] or(or(eq(list_partition_pruning.tcollist.a, 0), eq(list_partition_pruning.tcollist.a, 1)), or(eq(list_partition_pruning.tcollist.a, 6), eq(list_partition_pruning.tcollist.a, 11)))", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 120.00 root ", + "├─TableReader 40.00 root data:Selection", + "│ └─Selection 40.00 cop[tikv] or(or(eq(list_partition_pruning.tcollist.a, 0), eq(list_partition_pruning.tcollist.a, 1)), or(eq(list_partition_pruning.tcollist.a, 6), eq(list_partition_pruning.tcollist.a, 11)))", + "│ └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo", + "├─TableReader 40.00 root data:Selection", + "│ └─Selection 40.00 cop[tikv] or(or(eq(list_partition_pruning.tcollist.a, 0), eq(list_partition_pruning.tcollist.a, 1)), or(eq(list_partition_pruning.tcollist.a, 6), eq(list_partition_pruning.tcollist.a, 11)))", + "│ └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p2 keep order:false, stats:pseudo", + "└─TableReader 40.00 root data:Selection", + " └─Selection 40.00 cop[tikv] or(or(eq(list_partition_pruning.tcollist.a, 0), eq(list_partition_pruning.tcollist.a, 1)), or(eq(list_partition_pruning.tcollist.a, 6), eq(list_partition_pruning.tcollist.a, 11)))", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p3 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tlist where a in (0, 1, 2) and mod(a, 2)=0", + "DynamicPlan": [ + "TableReader 24.00 root partition:p0 data:Selection", + "└─Selection 24.00 cop[tikv] eq(mod(list_partition_pruning.tlist.a, 2), 0), in(list_partition_pruning.tlist.a, 0, 1, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 24.00 root data:Selection", + "└─Selection 24.00 cop[tikv] eq(mod(list_partition_pruning.tlist.a, 2), 0), in(list_partition_pruning.tlist.a, 0, 1, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tlist where a in (0, 3, 6) and a+1>3", + "DynamicPlan": [ + "TableReader 24.00 root partition:p0,p1,p2 data:Selection", + "└─Selection 24.00 cop[tikv] gt(plus(list_partition_pruning.tlist.a, 1), 3), in(list_partition_pruning.tlist.a, 0, 3, 6)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 72.00 root ", + "├─TableReader 24.00 root data:Selection", + "│ └─Selection 24.00 cop[tikv] gt(plus(list_partition_pruning.tlist.a, 1), 3), in(list_partition_pruning.tlist.a, 0, 3, 6)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo", + "├─TableReader 24.00 root data:Selection", + "│ └─Selection 24.00 cop[tikv] gt(plus(list_partition_pruning.tlist.a, 1), 3), in(list_partition_pruning.tlist.a, 0, 3, 6)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p1 keep order:false, stats:pseudo", + "└─TableReader 24.00 root data:Selection", + " └─Selection 24.00 cop[tikv] gt(plus(list_partition_pruning.tlist.a, 1), 3), in(list_partition_pruning.tlist.a, 0, 3, 6)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tcollist where a in (0, 1, 2) and mod(a, 2)=0", + "DynamicPlan": [ + "TableReader 24.00 root partition:p0 data:Selection", + "└─Selection 24.00 cop[tikv] eq(mod(list_partition_pruning.tcollist.a, 2), 0), in(list_partition_pruning.tcollist.a, 0, 1, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 24.00 root data:Selection", + "└─Selection 24.00 cop[tikv] eq(mod(list_partition_pruning.tcollist.a, 2), 0), in(list_partition_pruning.tcollist.a, 0, 1, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from tcollist where a in (0, 3, 6) and a+1>3", + "DynamicPlan": [ + "TableReader 24.00 root partition:p0,p1,p2 data:Selection", + "└─Selection 24.00 cop[tikv] gt(plus(list_partition_pruning.tcollist.a, 1), 3), in(list_partition_pruning.tcollist.a, 0, 3, 6)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 72.00 root ", + "├─TableReader 24.00 root data:Selection", + "│ └─Selection 24.00 cop[tikv] gt(plus(list_partition_pruning.tcollist.a, 1), 3), in(list_partition_pruning.tcollist.a, 0, 3, 6)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo", + "├─TableReader 24.00 root data:Selection", + "│ └─Selection 24.00 cop[tikv] gt(plus(list_partition_pruning.tcollist.a, 1), 3), in(list_partition_pruning.tcollist.a, 0, 3, 6)", + "│ └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p1 keep order:false, stats:pseudo", + "└─TableReader 24.00 root data:Selection", + " └─Selection 24.00 cop[tikv] gt(plus(list_partition_pruning.tcollist.a, 1), 3), in(list_partition_pruning.tcollist.a, 0, 3, 6)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' delete from tlist where a in (2)", + "DynamicPlan": [ + "Delete N/A root N/A", + "└─TableReader 10.00 root partition:p0 data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tlist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Delete N/A root N/A", + "└─TableReader 10.00 root data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tlist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' delete from tlist where a in (0, 5)", + "DynamicPlan": [ + "Delete N/A root N/A", + "└─TableReader 20.00 root partition:p0,p1 data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Delete N/A root N/A", + "└─PartitionUnion 40.00 root ", + " ├─TableReader 20.00 root data:Selection", + " │ └─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + " │ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo", + " └─TableReader 20.00 root data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' delete from tcollist where a in (2)", + "DynamicPlan": [ + "Delete N/A root N/A", + "└─TableReader 10.00 root partition:p0 data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tcollist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Delete N/A root N/A", + "└─TableReader 10.00 root data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tcollist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' delete from tcollist where a in (0, 5)", + "DynamicPlan": [ + "Delete N/A root N/A", + "└─TableReader 20.00 root partition:p0,p1 data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Delete N/A root N/A", + "└─PartitionUnion 40.00 root ", + " ├─TableReader 20.00 root data:Selection", + " │ └─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + " │ └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo", + " └─TableReader 20.00 root data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' update tlist set a=3 where a in (2)", + "DynamicPlan": [ + "Update N/A root N/A", + "└─TableReader 10.00 root partition:p0 data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tlist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Update N/A root N/A", + "└─TableReader 10.00 root data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tlist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' update tlist set a=3 where a in (0, 5)", + "DynamicPlan": [ + "Update N/A root N/A", + "└─TableReader 20.00 root partition:p0,p1 data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Update N/A root N/A", + "└─PartitionUnion 40.00 root ", + " ├─TableReader 20.00 root data:Selection", + " │ └─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + " │ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo", + " └─TableReader 20.00 root data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' update tcollist set a=3 where a in (2)", + "DynamicPlan": [ + "Update N/A root N/A", + "└─TableReader 10.00 root partition:p0 data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tcollist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Update N/A root N/A", + "└─TableReader 10.00 root data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tcollist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' update tcollist set a=3 where a in (0, 5)", + "DynamicPlan": [ + "Update N/A root N/A", + "└─TableReader 20.00 root partition:p0,p1 data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Update N/A root N/A", + "└─PartitionUnion 40.00 root ", + " ├─TableReader 20.00 root data:Selection", + " │ └─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + " │ └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo", + " └─TableReader 20.00 root data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' insert into tlist select * from tlist where a in (2)", + "DynamicPlan": [ + "Insert N/A root N/A", + "└─TableReader 10.00 root partition:p0 data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tlist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Insert N/A root N/A", + "└─TableReader 10.00 root data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tlist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' insert into tlist select * from tlist where a in (0, 5)", + "DynamicPlan": [ + "Insert N/A root N/A", + "└─TableReader 20.00 root partition:p0,p1 data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Insert N/A root N/A", + "└─PartitionUnion 40.00 root ", + " ├─TableReader 20.00 root data:Selection", + " │ └─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + " │ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo", + " └─TableReader 20.00 root data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' insert into tcollist select * from tcollist where a in (2)", + "DynamicPlan": [ + "Insert N/A root N/A", + "└─TableReader 10.00 root partition:p0 data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tcollist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Insert N/A root N/A", + "└─TableReader 10.00 root data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tcollist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' insert into tcollist select * from tcollist where a in (0, 5)", + "DynamicPlan": [ + "Insert N/A root N/A", + "└─TableReader 20.00 root partition:p0,p1 data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Insert N/A root N/A", + "└─PartitionUnion 40.00 root ", + " ├─TableReader 20.00 root data:Selection", + " │ └─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + " │ └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo", + " └─TableReader 20.00 root data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' replace tlist select * from tlist where a in (2)", + "DynamicPlan": [ + "Insert N/A root N/A", + "└─TableReader 10.00 root partition:p0 data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tlist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Insert N/A root N/A", + "└─TableReader 10.00 root data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tlist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' replace tlist select * from tlist where a in (0, 5)", + "DynamicPlan": [ + "Insert N/A root N/A", + "└─TableReader 20.00 root partition:p0,p1 data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Insert N/A root N/A", + "└─PartitionUnion 40.00 root ", + " ├─TableReader 20.00 root data:Selection", + " │ └─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + " │ └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo", + " └─TableReader 20.00 root data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' replace tcollist select * from tcollist where a in (2)", + "DynamicPlan": [ + "Insert N/A root N/A", + "└─TableReader 10.00 root partition:p0 data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tcollist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Insert N/A root N/A", + "└─TableReader 10.00 root data:Selection", + " └─Selection 10.00 cop[tikv] eq(list_partition_pruning.tcollist.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' replace tcollist select * from tcollist where a in (0, 5)", + "DynamicPlan": [ + "Insert N/A root N/A", + "└─TableReader 20.00 root partition:p0,p1 data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Insert N/A root N/A", + "└─PartitionUnion 40.00 root ", + " ├─TableReader 20.00 root data:Selection", + " │ └─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + " │ └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo", + " └─TableReader 20.00 root data:Selection", + " └─Selection 20.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 5)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p1 keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestPartitionTableExplain", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t", + "DynamicPlan": [ + "TableReader 3.00 root partition:all data:TableFullScan", + "└─TableFullScan 3.00 cop[tikv] table:t keep order:false" + ], + "StaticPlan": [ + "PartitionUnion 3.00 root ", + "├─TableReader 1.00 root data:TableFullScan", + "│ └─TableFullScan 1.00 cop[tikv] table:t, partition:P0 keep order:false", + "├─TableReader 1.00 root data:TableFullScan", + "│ └─TableFullScan 1.00 cop[tikv] table:t, partition:p1 keep order:false", + "└─TableReader 1.00 root data:TableFullScan", + " └─TableFullScan 1.00 cop[tikv] table:t, partition:P2 keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t partition(P0,p1)", + "DynamicPlan": [ + "TableReader 3.00 root partition:P0,p1 data:TableFullScan", + "└─TableFullScan 3.00 cop[tikv] table:t keep order:false" + ], + "StaticPlan": [ + "PartitionUnion 2.00 root ", + "├─TableReader 1.00 root data:TableFullScan", + "│ └─TableFullScan 1.00 cop[tikv] table:t, partition:P0 keep order:false", + "└─TableReader 1.00 root data:TableFullScan", + " └─TableFullScan 1.00 cop[tikv] table:t, partition:p1 keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a = 1", + "DynamicPlan": [ + "Point_Get 1.00 root table:t, partition:p1 handle:1" + ], + "StaticPlan": [ + "Point_Get 1.00 root table:t, partition:p1 handle:1" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a = 2", + "DynamicPlan": [ + "Point_Get 1.00 root table:t, partition:P2 handle:2" + ], + "StaticPlan": [ + "Point_Get 1.00 root table:t, partition:P2 handle:2" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a = 1 OR a = 2", + "DynamicPlan": [ + "TableReader 2.00 root partition:p1,P2 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:t range:[1,1], [2,2], keep order:false" + ], + "StaticPlan": [ + "PartitionUnion 2.00 root ", + "├─Batch_Point_Get 1.00 root table:t, partition:p1 handle:[1], keep order:false, desc:false", + "└─Batch_Point_Get 1.00 root table:t, partition:P2 handle:[2], keep order:false, desc:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a IN (2,3,4)", + "DynamicPlan": [ + "Batch_Point_Get 3.00 root table:t, partition:P0,p1,P2 handle:[2 3 4], keep order:false, desc:false" + ], + "StaticPlan": [ + "Batch_Point_Get 3.00 root table:t, partition:P0,p1,P2 handle:[2 3 4], keep order:false, desc:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a IN (2,3)", + "DynamicPlan": [ + "Batch_Point_Get 2.00 root table:t, partition:P0,P2 handle:[2 3], keep order:false, desc:false" + ], + "StaticPlan": [ + "Batch_Point_Get 2.00 root table:t, partition:P0,P2 handle:[2 3], keep order:false, desc:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b = 1", + "DynamicPlan": [ + "IndexReader 1.00 root partition:all index:IndexRangeScan", + "└─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false" + ], + "StaticPlan": [ + "PartitionUnion 3.00 root ", + "├─IndexReader 1.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, partition:P0, index:b(b) range:[1,1], keep order:false", + "├─IndexReader 1.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, partition:p1, index:b(b) range:[1,1], keep order:false", + "└─IndexReader 1.00 root index:IndexRangeScan", + " └─IndexRangeScan 1.00 cop[tikv] table:t, partition:P2, index:b(b) range:[1,1], keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b = 2", + "DynamicPlan": [ + "IndexReader 1.00 root partition:all index:IndexRangeScan", + "└─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[2,2], keep order:false" + ], + "StaticPlan": [ + "PartitionUnion 3.00 root ", + "├─IndexReader 1.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, partition:P0, index:b(b) range:[2,2], keep order:false", + "├─IndexReader 1.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, partition:p1, index:b(b) range:[2,2], keep order:false", + "└─IndexReader 1.00 root index:IndexRangeScan", + " └─IndexRangeScan 1.00 cop[tikv] table:t, partition:P2, index:b(b) range:[2,2], keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b = 1 OR b = 2", + "DynamicPlan": [ + "IndexReader 2.00 root partition:all index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:t, index:b(b) range:[1,2], keep order:false" + ], + "StaticPlan": [ + "PartitionUnion 3.00 root ", + "├─IndexReader 1.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, partition:P0, index:b(b) range:[1,2], keep order:false", + "├─IndexReader 1.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, partition:p1, index:b(b) range:[1,2], keep order:false", + "└─IndexReader 1.00 root index:IndexRangeScan", + " └─IndexRangeScan 1.00 cop[tikv] table:t, partition:P2, index:b(b) range:[1,2], keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b IN (2,3,4)", + "DynamicPlan": [ + "IndexReader 2.00 root partition:all index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:t, index:b(b) range:[2,2], [3,3], [4,4], keep order:false" + ], + "StaticPlan": [ + "PartitionUnion 3.00 root ", + "├─IndexReader 1.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, partition:P0, index:b(b) range:[2,2], [3,3], [4,4], keep order:false", + "├─IndexReader 1.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, partition:p1, index:b(b) range:[2,2], [3,3], [4,4], keep order:false", + "└─IndexReader 1.00 root index:IndexRangeScan", + " └─IndexRangeScan 1.00 cop[tikv] table:t, partition:P2, index:b(b) range:[2,2], [3,3], [4,4], keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b IN (2,3)", + "DynamicPlan": [ + "IndexReader 2.00 root partition:all index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:t, index:b(b) range:[2,2], [3,3], keep order:false" + ], + "StaticPlan": [ + "PartitionUnion 3.00 root ", + "├─IndexReader 1.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, partition:P0, index:b(b) range:[2,2], [3,3], keep order:false", + "├─IndexReader 1.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, partition:p1, index:b(b) range:[2,2], [3,3], keep order:false", + "└─IndexReader 1.00 root index:IndexRangeScan", + " └─IndexRangeScan 1.00 cop[tikv] table:t, partition:P2, index:b(b) range:[2,2], [3,3], keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t,t2 where t2.a = 1 and t2.b = t.b", + "DynamicPlan": [ + "Projection 1.00 root test.t.a, test.t.b, test.t2.a, test.t2.b", + "└─HashJoin 1.00 root inner join, equal:[eq(test.t2.b, test.t.b)]", + " ├─TableReader(Build) 1.00 root data:Selection", + " │ └─Selection 1.00 cop[tikv] eq(test.t2.a, 1), not(isnull(test.t2.b))", + " │ └─TableFullScan 3.00 cop[tikv] table:t2 keep order:false", + " └─IndexReader(Probe) 3.00 root partition:all index:IndexFullScan", + " └─IndexFullScan 3.00 cop[tikv] table:t, index:b(b) keep order:false" + ], + "StaticPlan": [ + "Projection 1.00 root test.t.a, test.t.b, test.t2.a, test.t2.b", + "└─HashJoin 1.00 root inner join, equal:[eq(test.t2.b, test.t.b)]", + " ├─TableReader(Build) 1.00 root data:Selection", + " │ └─Selection 1.00 cop[tikv] eq(test.t2.a, 1), not(isnull(test.t2.b))", + " │ └─TableFullScan 3.00 cop[tikv] table:t2 keep order:false", + " └─PartitionUnion(Probe) 3.00 root ", + " ├─IndexReader 1.00 root index:IndexFullScan", + " │ └─IndexFullScan 1.00 cop[tikv] table:t, partition:P0, index:b(b) keep order:false", + " ├─IndexReader 1.00 root index:IndexFullScan", + " │ └─IndexFullScan 1.00 cop[tikv] table:t, partition:p1, index:b(b) keep order:false", + " └─IndexReader 1.00 root index:IndexFullScan", + " └─IndexFullScan 1.00 cop[tikv] table:t, partition:P2, index:b(b) keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t partition (p1),t2 where t2.a = 1 and t2.b = t.b", + "DynamicPlan": [ + "Projection 1.00 root test.t.a, test.t.b, test.t2.a, test.t2.b", + "└─HashJoin 1.00 root inner join, equal:[eq(test.t2.b, test.t.b)]", + " ├─TableReader(Build) 1.00 root data:Selection", + " │ └─Selection 1.00 cop[tikv] eq(test.t2.a, 1), not(isnull(test.t2.b))", + " │ └─TableFullScan 3.00 cop[tikv] table:t2 keep order:false", + " └─IndexReader(Probe) 3.00 root partition:p1 index:IndexFullScan", + " └─IndexFullScan 3.00 cop[tikv] table:t, index:b(b) keep order:false" + ], + "StaticPlan": [ + "HashJoin 1.00 root inner join, equal:[eq(test.t.b, test.t2.b)]", + "├─TableReader(Build) 1.00 root data:Selection", + "│ └─Selection 1.00 cop[tikv] eq(test.t2.a, 1), not(isnull(test.t2.b))", + "│ └─TableFullScan 3.00 cop[tikv] table:t2 keep order:false", + "└─IndexReader(Probe) 1.00 root index:IndexFullScan", + " └─IndexFullScan 1.00 cop[tikv] table:t, partition:p1, index:b(b) keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t,t2 where t2.a = 1 and t2.b = t.b and t.a = 1", + "DynamicPlan": [ + "HashJoin 1.00 root inner join, equal:[eq(test.t.b, test.t2.b)]", + "├─Selection(Build) 1.00 root not(isnull(test.t.b))", + "│ └─Point_Get 1.00 root table:t, partition:p1 handle:1", + "└─TableReader(Probe) 1.00 root data:Selection", + " └─Selection 1.00 cop[tikv] eq(test.t2.a, 1), not(isnull(test.t2.b))", + " └─TableFullScan 3.00 cop[tikv] table:t2 keep order:false" + ], + "StaticPlan": [ + "HashJoin 1.00 root inner join, equal:[eq(test.t.b, test.t2.b)]", + "├─Selection(Build) 1.00 root not(isnull(test.t.b))", + "│ └─Point_Get 1.00 root table:t, partition:p1 handle:1", + "└─TableReader(Probe) 1.00 root data:Selection", + " └─Selection 1.00 cop[tikv] eq(test.t2.a, 1), not(isnull(test.t2.b))", + " └─TableFullScan 3.00 cop[tikv] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t partition (P0,p1) where b = 1", + "DynamicPlan": [ + "IndexReader 1.00 root partition:P0,p1 index:IndexRangeScan", + "└─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false" + ], + "StaticPlan": [ + "PartitionUnion 2.00 root ", + "├─IndexReader 1.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, partition:P0, index:b(b) range:[1,1], keep order:false", + "└─IndexReader 1.00 root index:IndexRangeScan", + " └─IndexRangeScan 1.00 cop[tikv] table:t, partition:p1, index:b(b) range:[1,1], keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t partition (p1,P2) where b = 1 OR b = 2", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p1,P2 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:t, index:b(b) range:[1,2], keep order:false" + ], + "StaticPlan": [ + "PartitionUnion 2.00 root ", + "├─IndexReader 1.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, partition:p1, index:b(b) range:[1,2], keep order:false", + "└─IndexReader 1.00 root index:IndexRangeScan", + " └─IndexRangeScan 1.00 cop[tikv] table:t, partition:P2, index:b(b) range:[1,2], keep order:false" + ] + } + ] + }, + { + "Name": "TestBatchPointGetTablePartition", + "Cases": [ + { + "SQL": "select * from thash1 where a in (1,2) and b = 1", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p1 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:thash1, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Batch_Point_Get 2.00 root table:thash1, partition:p1, index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Result": [ + "1 1", + "2 1" + ] + }, + { + "SQL": "select * from thash1 where a in (1,2) and b = 1 order by a", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p1 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:thash1, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "Batch_Point_Get 2.00 root table:thash1, partition:p1, index:PRIMARY(a, b) keep order:true, desc:false" + ], + "Result": [ + "1 1", + "2 1" + ] + }, + { + "SQL": "select * from thash1 where a in (1,2) and b = 1 order by a desc", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p1 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:thash1, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:true, desc, stats:pseudo" + ], + "StaticPlan": [ + "Batch_Point_Get 2.00 root table:thash1, partition:p1, index:PRIMARY(a, b) keep order:true, desc:true" + ], + "Result": [ + "2 1", + "1 1" + ] + }, + { + "SQL": "select * from thash1 where a = 1 and b in (1,2)", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p0,p1 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:thash1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 4.00 root ", + "├─Batch_Point_Get 2.00 root table:thash1, partition:p0, index:PRIMARY(a, b) keep order:false, desc:false", + "└─Batch_Point_Get 2.00 root table:thash1, partition:p1, index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Result": [ + "1 1", + "1 2" + ] + }, + { + "SQL": "select * from thash1 where a = 1 and b in (1,2) order by b", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p0,p1 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:thash1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "Sort 4.00 root test.thash1.b", + "└─PartitionUnion 4.00 root ", + " ├─Batch_Point_Get 2.00 root table:thash1, partition:p0, index:PRIMARY(a, b) keep order:false, desc:false", + " └─Batch_Point_Get 2.00 root table:thash1, partition:p1, index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Result": [ + "1 1", + "1 2" + ] + }, + { + "SQL": "select * from thash1 where a = 1 and b in (1,2) order by b desc", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p0,p1 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:thash1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:true, desc, stats:pseudo" + ], + "StaticPlan": [ + "Sort 4.00 root test.thash1.b:desc", + "└─PartitionUnion 4.00 root ", + " ├─Batch_Point_Get 2.00 root table:thash1, partition:p0, index:PRIMARY(a, b) keep order:false, desc:false", + " └─Batch_Point_Get 2.00 root table:thash1, partition:p1, index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Result": [ + "1 2", + "1 1" + ] + }, + { + "SQL": "select * from trange1 where a in (1,2) and b = 1", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p0 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:trange1, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "IndexReader 2.00 root index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:trange1, partition:p0, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "2 1" + ] + }, + { + "SQL": "select * from trange1 where a in (1,2) and b = 1 order by a", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p0 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:trange1, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "IndexReader 2.00 root index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:trange1, partition:p0, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:true, stats:pseudo" + ], + "Result": [ + "1 1", + "2 1" + ] + }, + { + "SQL": "select * from trange1 where a in (1,2) and b = 1 order by a desc", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p0 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:trange1, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:true, desc, stats:pseudo" + ], + "StaticPlan": [ + "IndexReader 2.00 root index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:trange1, partition:p0, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:true, desc, stats:pseudo" + ], + "Result": [ + "2 1", + "1 1" + ] + }, + { + "SQL": "select * from trange1 where a = 1 and b in (1,2)", + "DynamicPlan": [ + "IndexReader 2.00 root partition:all index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:trange1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 4.00 root ", + "├─IndexReader 2.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 2.00 cop[tikv] table:trange1, partition:p0, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo", + "└─IndexReader 2.00 root index:IndexRangeScan", + " └─IndexRangeScan 2.00 cop[tikv] table:trange1, partition:p1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "1 2" + ] + }, + { + "SQL": "select * from trange1 where a = 1 and b in (1,2) order by b", + "DynamicPlan": [ + "IndexReader 2.00 root partition:all index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:trange1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "Sort 4.00 root test.trange1.b", + "└─PartitionUnion 4.00 root ", + " ├─IndexReader 2.00 root index:IndexRangeScan", + " │ └─IndexRangeScan 2.00 cop[tikv] table:trange1, partition:p0, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo", + " └─IndexReader 2.00 root index:IndexRangeScan", + " └─IndexRangeScan 2.00 cop[tikv] table:trange1, partition:p1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "1 2" + ] + }, + { + "SQL": "select * from trange1 where a = 1 and b in (1,2) order by b desc", + "DynamicPlan": [ + "IndexReader 2.00 root partition:all index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:trange1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:true, desc, stats:pseudo" + ], + "StaticPlan": [ + "Sort 4.00 root test.trange1.b:desc", + "└─PartitionUnion 4.00 root ", + " ├─IndexReader 2.00 root index:IndexRangeScan", + " │ └─IndexRangeScan 2.00 cop[tikv] table:trange1, partition:p0, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo", + " └─IndexReader 2.00 root index:IndexRangeScan", + " └─IndexRangeScan 2.00 cop[tikv] table:trange1, partition:p1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 2", + "1 1" + ] + }, + { + "SQL": "select * from tlist1 where a in (1,2) and b = 1", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p0 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:tlist1, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "IndexReader 2.00 root index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:tlist1, partition:p0, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "2 1" + ] + }, + { + "SQL": "select * from tlist1 where a in (1,2) and b = 1 order by a", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p0 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:tlist1, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "IndexReader 2.00 root index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:tlist1, partition:p0, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:true, stats:pseudo" + ], + "Result": [ + "1 1", + "2 1" + ] + }, + { + "SQL": "select * from tlist1 where a in (1,2) and b = 1 order by a desc", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p0 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:tlist1, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:true, desc, stats:pseudo" + ], + "StaticPlan": [ + "IndexReader 2.00 root index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:tlist1, partition:p0, index:PRIMARY(a, b) range:[1 1,1 1], [2 1,2 1], keep order:true, desc, stats:pseudo" + ], + "Result": [ + "2 1", + "1 1" + ] + }, + { + "SQL": "select * from tlist1 where a = 1 and b in (1,2)", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p0,p1 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:tlist1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 4.00 root ", + "├─IndexReader 2.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 2.00 cop[tikv] table:tlist1, partition:p0, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo", + "└─IndexReader 2.00 root index:IndexRangeScan", + " └─IndexRangeScan 2.00 cop[tikv] table:tlist1, partition:p1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "1 2" + ] + }, + { + "SQL": "select * from tlist1 where a = 1 and b in (1,2) order by b", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p0,p1 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:tlist1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "Sort 4.00 root test.tlist1.b", + "└─PartitionUnion 4.00 root ", + " ├─IndexReader 2.00 root index:IndexRangeScan", + " │ └─IndexRangeScan 2.00 cop[tikv] table:tlist1, partition:p0, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo", + " └─IndexReader 2.00 root index:IndexRangeScan", + " └─IndexRangeScan 2.00 cop[tikv] table:tlist1, partition:p1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "1 2" + ] + }, + { + "SQL": "select * from tlist1 where a = 1 and b in (1,2) order by b desc", + "DynamicPlan": [ + "IndexReader 2.00 root partition:p0,p1 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:tlist1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:true, desc, stats:pseudo" + ], + "StaticPlan": [ + "Sort 4.00 root test.tlist1.b:desc", + "└─PartitionUnion 4.00 root ", + " ├─IndexReader 2.00 root index:IndexRangeScan", + " │ └─IndexRangeScan 2.00 cop[tikv] table:tlist1, partition:p0, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo", + " └─IndexReader 2.00 root index:IndexRangeScan", + " └─IndexRangeScan 2.00 cop[tikv] table:tlist1, partition:p1, index:PRIMARY(a, b) range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 2", + "1 1" + ] + }, + { + "SQL": "select * from thash2 where a in (1,2) and b = 1", + "DynamicPlan": [ + "TableReader 2.00 root partition:p1 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:thash2 range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Batch_Point_Get 2.00 root table:thash2, partition:p1, clustered index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Result": [ + "1 1", + "2 1" + ] + }, + { + "SQL": "select * from thash2 where a in (1,2) and b = 1 order by a", + "DynamicPlan": [ + "TableReader 2.00 root partition:p1 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:thash2 range:[1 1,1 1], [2 1,2 1], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "Batch_Point_Get 2.00 root table:thash2, partition:p1, clustered index:PRIMARY(a, b) keep order:true, desc:false" + ], + "Result": [ + "1 1", + "2 1" + ] + }, + { + "SQL": "select * from thash2 where a in (1,2) and b = 1 order by a desc", + "DynamicPlan": [ + "Sort 0.02 root test.thash2.a:desc", + "└─TableReader 2.00 root partition:p1 data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:thash2 range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Batch_Point_Get 2.00 root table:thash2, partition:p1, clustered index:PRIMARY(a, b) keep order:true, desc:true" + ], + "Result": [ + "2 1", + "1 1" + ] + }, + { + "SQL": "select * from thash2 where a = 1 and b in (1,2)", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0,p1 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:thash2 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 0.04 root ", + "├─Batch_Point_Get 2.00 root table:thash2, partition:p0, clustered index:PRIMARY(a, b) keep order:false, desc:false", + "└─Batch_Point_Get 2.00 root table:thash2, partition:p1, clustered index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Result": [ + "1 1", + "1 2" + ] + }, + { + "SQL": "select * from thash2 where a = 1 and b in (1,2) order by b", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0,p1 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:thash2 range:[1 1,1 1], [1 2,1 2], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "Sort 0.04 root test.thash2.b", + "└─PartitionUnion 0.04 root ", + " ├─Batch_Point_Get 2.00 root table:thash2, partition:p0, clustered index:PRIMARY(a, b) keep order:false, desc:false", + " └─Batch_Point_Get 2.00 root table:thash2, partition:p1, clustered index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Result": [ + "1 1", + "1 2" + ] + }, + { + "SQL": "select * from thash2 where a = 1 and b in (1,2) order by b desc", + "DynamicPlan": [ + "Sort 0.02 root test.thash2.b:desc", + "└─TableReader 2.00 root partition:p0,p1 data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:thash2 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Sort 0.04 root test.thash2.b:desc", + "└─PartitionUnion 0.04 root ", + " ├─Batch_Point_Get 2.00 root table:thash2, partition:p0, clustered index:PRIMARY(a, b) keep order:false, desc:false", + " └─Batch_Point_Get 2.00 root table:thash2, partition:p1, clustered index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Result": [ + "1 2", + "1 1" + ] + }, + { + "SQL": "select * from trange2 where a in (1,2) and b = 1", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange2 range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 2.00 root data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange2, partition:p0 range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "2 1" + ] + }, + { + "SQL": "select * from trange2 where a in (1,2) and b = 1 order by a", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange2 range:[1 1,1 1], [2 1,2 1], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 2.00 root data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange2, partition:p0 range:[1 1,1 1], [2 1,2 1], keep order:true, stats:pseudo" + ], + "Result": [ + "1 1", + "2 1" + ] + }, + { + "SQL": "select * from trange2 where a in (1,2) and b = 1 order by a desc", + "DynamicPlan": [ + "Sort 0.02 root test.trange2.a:desc", + "└─TableReader 2.00 root partition:p0 data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:trange2 range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Sort 0.02 root test.trange2.a:desc", + "└─TableReader 2.00 root data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:trange2, partition:p0 range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "Result": [ + "2 1", + "1 1" + ] + }, + { + "SQL": "select * from trange2 where a = 1 and b in (1,2)", + "DynamicPlan": [ + "TableReader 2.00 root partition:all data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange2 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 0.04 root ", + "├─TableReader 2.00 root data:TableRangeScan", + "│ └─TableRangeScan 2.00 cop[tikv] table:trange2, partition:p0 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo", + "└─TableReader 2.00 root data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:trange2, partition:p1 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "1 2" + ] + }, + { + "SQL": "select * from trange2 where a = 1 and b in (1,2) order by b", + "DynamicPlan": [ + "TableReader 2.00 root partition:all data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange2 range:[1 1,1 1], [1 2,1 2], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "Sort 0.04 root test.trange2.b", + "└─PartitionUnion 0.04 root ", + " ├─TableReader 2.00 root data:TableRangeScan", + " │ └─TableRangeScan 2.00 cop[tikv] table:trange2, partition:p0 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo", + " └─TableReader 2.00 root data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:trange2, partition:p1 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "1 2" + ] + }, + { + "SQL": "select * from trange2 where a = 1 and b in (1,2) order by b desc", + "DynamicPlan": [ + "Sort 0.02 root test.trange2.b:desc", + "└─TableReader 2.00 root partition:all data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:trange2 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Sort 0.04 root test.trange2.b:desc", + "└─PartitionUnion 0.04 root ", + " ├─TableReader 2.00 root data:TableRangeScan", + " │ └─TableRangeScan 2.00 cop[tikv] table:trange2, partition:p0 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo", + " └─TableReader 2.00 root data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:trange2, partition:p1 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 2", + "1 1" + ] + }, + { + "SQL": "select * from tlist2 where a in (1,2) and b = 1", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist2 range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 2.00 root data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist2, partition:p0 range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "2 1" + ] + }, + { + "SQL": "select * from tlist2 where a in (1,2) and b = 1 order by a", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist2 range:[1 1,1 1], [2 1,2 1], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 2.00 root data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist2, partition:p0 range:[1 1,1 1], [2 1,2 1], keep order:true, stats:pseudo" + ], + "Result": [ + "1 1", + "2 1" + ] + }, + { + "SQL": "select * from tlist2 where a in (1,2) and b = 1 order by a desc", + "DynamicPlan": [ + "Sort 0.02 root test.tlist2.a:desc", + "└─TableReader 2.00 root partition:p0 data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:tlist2 range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Sort 0.02 root test.tlist2.a:desc", + "└─TableReader 2.00 root data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:tlist2, partition:p0 range:[1 1,1 1], [2 1,2 1], keep order:false, stats:pseudo" + ], + "Result": [ + "2 1", + "1 1" + ] + }, + { + "SQL": "select * from tlist2 where a = 1 and b in (1,2)", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0,p1 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist2 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 0.04 root ", + "├─TableReader 2.00 root data:TableRangeScan", + "│ └─TableRangeScan 2.00 cop[tikv] table:tlist2, partition:p0 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo", + "└─TableReader 2.00 root data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:tlist2, partition:p1 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "1 2" + ] + }, + { + "SQL": "select * from tlist2 where a = 1 and b in (1,2) order by b", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0,p1 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist2 range:[1 1,1 1], [1 2,1 2], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "Sort 0.04 root test.tlist2.b", + "└─PartitionUnion 0.04 root ", + " ├─TableReader 2.00 root data:TableRangeScan", + " │ └─TableRangeScan 2.00 cop[tikv] table:tlist2, partition:p0 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo", + " └─TableReader 2.00 root data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:tlist2, partition:p1 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "1 2" + ] + }, + { + "SQL": "select * from tlist2 where a = 1 and b in (1,2) order by b desc", + "DynamicPlan": [ + "Sort 0.02 root test.tlist2.b:desc", + "└─TableReader 2.00 root partition:p0,p1 data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:tlist2 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Sort 0.04 root test.tlist2.b:desc", + "└─PartitionUnion 0.04 root ", + " ├─TableReader 2.00 root data:TableRangeScan", + " │ └─TableRangeScan 2.00 cop[tikv] table:tlist2, partition:p0 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo", + " └─TableReader 2.00 root data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:tlist2, partition:p1 range:[1 1,1 1], [1 2,1 2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 2", + "1 1" + ] + }, + { + "SQL": "select * from thash3 where a in (1,2) and 1 = 1", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0,p1 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:thash3 range:[1,1], [2,2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 4.00 root ", + "├─Batch_Point_Get 2.00 root table:thash3, partition:p0 handle:[2], keep order:false, desc:false", + "└─Batch_Point_Get 2.00 root table:thash3, partition:p1 handle:[1], keep order:false, desc:false" + ], + "Result": [ + "1 0", + "2 0" + ] + }, + { + "SQL": "select * from thash3 where a in (1,3) and 1 = 1", + "DynamicPlan": [ + "TableReader 2.00 root partition:p1 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:thash3 range:[1,1], [3,3], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Batch_Point_Get 2.00 root table:thash3, partition:p1 handle:[1 3], keep order:false, desc:false" + ], + "Result": [ + "1 0", + "3 0" + ] + }, + { + "SQL": "select * from thash3 where a in (1,3) and 1 = 1 order by a", + "DynamicPlan": [ + "TableReader 2.00 root partition:p1 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:thash3 range:[1,1], [3,3], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "Batch_Point_Get 2.00 root table:thash3, partition:p1 handle:[1 3], keep order:true, desc:false" + ], + "Result": [ + "1 0", + "3 0" + ] + }, + { + "SQL": "select * from thash3 where a in (1,3) and 1 = 1 order by a desc", + "DynamicPlan": [ + "TableReader 2.00 root partition:p1 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:thash3 range:[1,1], [3,3], keep order:true, desc, stats:pseudo" + ], + "StaticPlan": [ + "Batch_Point_Get 2.00 root table:thash3, partition:p1 handle:[1 3], keep order:true, desc:true" + ], + "Result": [ + "3 0", + "1 0" + ] + }, + { + "SQL": "select * from thash3 partition(p0) where a in (1,4)", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:thash3 range:[1,1], [4,4], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Batch_Point_Get 2.00 root table:thash3, partition:p0 handle:[4], keep order:false, desc:false" + ], + "Result": [ + "4 0" + ] + }, + { + "SQL": "select * from thash3 partition(p1) where a in (2,4)", + "DynamicPlan": [ + "TableReader 2.00 root partition:dual data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:thash3 range:[2,2], [4,4], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableDual 0.00 root rows:0" + ], + "Result": null + }, + { + "SQL": "select * from thash3 partition(p0,p1) where a in (2,4)", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:thash3 range:[2,2], [4,4], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Batch_Point_Get 2.00 root table:thash3, partition:p0 handle:[2 4], keep order:false, desc:false" + ], + "Result": [ + "2 0", + "4 0" + ] + }, + { + "SQL": "select * from trange3 where a in (1,2) and 1 = 1", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange3 range:[1,1], [2,2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 2.00 root data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange3, partition:p0 range:[1,1], [2,2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 0", + "2 0" + ] + }, + { + "SQL": "select * from trange3 where a in (1,3) and 1 = 1", + "DynamicPlan": [ + "TableReader 2.00 root partition:all data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange3 range:[1,1], [3,3], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 4.00 root ", + "├─TableReader 2.00 root data:TableRangeScan", + "│ └─TableRangeScan 2.00 cop[tikv] table:trange3, partition:p0 range:[1,1], [3,3], keep order:false, stats:pseudo", + "└─TableReader 2.00 root data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:trange3, partition:p1 range:[1,1], [3,3], keep order:false, stats:pseudo" + ], + "Result": [ + "1 0", + "3 0" + ] + }, + { + "SQL": "select * from trange3 where a in (1,3) and 1 = 1 order by a", + "DynamicPlan": [ + "TableReader 2.00 root partition:all data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange3 range:[1,1], [3,3], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "Sort 4.00 root test.trange3.a", + "└─PartitionUnion 4.00 root ", + " ├─TableReader 2.00 root data:TableRangeScan", + " │ └─TableRangeScan 2.00 cop[tikv] table:trange3, partition:p0 range:[1,1], [3,3], keep order:false, stats:pseudo", + " └─TableReader 2.00 root data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:trange3, partition:p1 range:[1,1], [3,3], keep order:false, stats:pseudo" + ], + "Result": [ + "1 0", + "3 0" + ] + }, + { + "SQL": "select * from trange3 where a in (1,3) and 1 = 1 order by a desc", + "DynamicPlan": [ + "TableReader 2.00 root partition:all data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange3 range:[1,1], [3,3], keep order:true, desc, stats:pseudo" + ], + "StaticPlan": [ + "Sort 4.00 root test.trange3.a:desc", + "└─PartitionUnion 4.00 root ", + " ├─TableReader 2.00 root data:TableRangeScan", + " │ └─TableRangeScan 2.00 cop[tikv] table:trange3, partition:p0 range:[1,1], [3,3], keep order:false, stats:pseudo", + " └─TableReader 2.00 root data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:trange3, partition:p1 range:[1,1], [3,3], keep order:false, stats:pseudo" + ], + "Result": [ + "3 0", + "1 0" + ] + }, + { + "SQL": "select * from trange3 partition(p0) where a in (1,4)", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange3 range:[1,1], [4,4], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 2.00 root data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange3, partition:p0 range:[1,1], [4,4], keep order:false, stats:pseudo" + ], + "Result": [ + "1 0" + ] + }, + { + "SQL": "select * from trange3 partition(p1) where a in (1,2)", + "DynamicPlan": [ + "TableReader 2.00 root partition:dual data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange3 range:[1,1], [2,2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableDual 0.00 root rows:0" + ], + "Result": null + }, + { + "SQL": "select * from trange3 partition(p0,p1) where a in (1,2)", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange3 range:[1,1], [2,2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 2.00 root data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:trange3, partition:p0 range:[1,1], [2,2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 0", + "2 0" + ] + }, + { + "SQL": "select * from tlist3 where a in (1,2) and 1 = 1", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist3 range:[1,1], [2,2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 2.00 root data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist3, partition:p0 range:[1,1], [2,2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 0", + "2 0" + ] + }, + { + "SQL": "select * from tlist3 where a in (1,3) and 1 = 1", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0,p1 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist3 range:[1,1], [3,3], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "PartitionUnion 4.00 root ", + "├─TableReader 2.00 root data:TableRangeScan", + "│ └─TableRangeScan 2.00 cop[tikv] table:tlist3, partition:p0 range:[1,1], [3,3], keep order:false, stats:pseudo", + "└─TableReader 2.00 root data:TableRangeScan", + " └─TableRangeScan 2.00 cop[tikv] table:tlist3, partition:p1 range:[1,1], [3,3], keep order:false, stats:pseudo" + ], + "Result": [ + "1 0", + "3 0" + ] + }, + { + "SQL": "select * from tlist3 where a in (1,2) and 1 = 1 order by a", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist3 range:[1,1], [2,2], keep order:true, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 2.00 root data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist3, partition:p0 range:[1,1], [2,2], keep order:true, stats:pseudo" + ], + "Result": [ + "1 0", + "2 0" + ] + }, + { + "SQL": "select * from tlist3 where a in (1,2) and 1 = 1 order by a desc", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist3 range:[1,1], [2,2], keep order:true, desc, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 2.00 root data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist3, partition:p0 range:[1,1], [2,2], keep order:true, desc, stats:pseudo" + ], + "Result": [ + "2 0", + "1 0" + ] + }, + { + "SQL": "select * from tlist3 partition(p0) where a in (1,4)", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist3 range:[1,1], [4,4], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 2.00 root data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist3, partition:p0 range:[1,1], [4,4], keep order:false, stats:pseudo" + ], + "Result": [ + "1 0" + ] + }, + { + "SQL": "select * from tlist3 partition(p1) where a in (1,2)", + "DynamicPlan": [ + "TableReader 2.00 root partition:dual data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist3 range:[1,1], [2,2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableDual 0.00 root rows:0" + ], + "Result": null + }, + { + "SQL": "select * from tlist3 partition(p0,p1) where a in (1,2)", + "DynamicPlan": [ + "TableReader 2.00 root partition:p0 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist3 range:[1,1], [2,2], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "TableReader 2.00 root data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:tlist3, partition:p0 range:[1,1], [2,2], keep order:false, stats:pseudo" + ], + "Result": [ + "1 0", + "2 0" + ] + }, + { + "SQL": "select _tidb_rowid, a from issue45889 where _tidb_rowid in (7, 8)", + "DynamicPlan": [ + "Projection 8000.00 root test.issue45889._tidb_rowid, test.issue45889.a", + "└─TableReader 10000.00 root partition:all data:TableRangeScan", + " └─TableRangeScan 10000.00 cop[tikv] table:issue45889 range:[7,7], [8,8], keep order:false, stats:pseudo" + ], + "StaticPlan": [ + "Projection 16000.00 root test.issue45889._tidb_rowid, test.issue45889.a", + "└─PartitionUnion 16000.00 root ", + " ├─TableReader 10000.00 root data:TableRangeScan", + " │ └─TableRangeScan 10000.00 cop[tikv] table:issue45889, partition:p0 range:[7,7], [8,8], keep order:false, stats:pseudo", + " └─TableReader 10000.00 root data:TableRangeScan", + " └─TableRangeScan 10000.00 cop[tikv] table:issue45889, partition:p1 range:[7,7], [8,8], keep order:false, stats:pseudo" + ], + "Result": [ + "7 3", + "8 3" + ] + } + ] + }, + { + "Name": "TestBatchPointGetPartitionForAccessObject", + "Cases": [ + { + "SQL": "explain format='brief' select * from t1 where b in (1, 2)", + "Plan": [ + "Batch_Point_Get 2.00 root table:t1, partition:p1,p2, index:b(b) keep order:false, desc:false" + ] + }, + { + "SQL": "explain format='brief' select * from t1 where b in (1, 2, 1)", + "Plan": [ + "Batch_Point_Get 3.00 root table:t1, partition:p1,p2, index:b(b) keep order:false, desc:false" + ] + }, + { + "SQL": "explain format='brief' select * from t2 where id in (1, 3)", + "Plan": [ + "Batch_Point_Get 2.00 root table:t2, partition:p0,p1 handle:[1 3], keep order:false, desc:false" + ] + }, + { + "SQL": "explain format='brief' select * from t3 where id in (1, 3)", + "Plan": [ + "TableReader 2.00 root partition:p0,p1 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:t3 range:[1,1], [3,3], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format='brief' select * from t4 where (id, name_id) in ((1, 1), (3, 3))", + "Plan": [ + "IndexReader 2.00 root partition:p0,p1 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:t4, index:id(id, name_id) range:[1 1,1 1], [3 3,3 3], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format='brief' select * from t5 where (id, name) in ((1, 'a'), (3, 'c'))", + "Plan": [ + "IndexReader 2.00 root partition:p0,p1 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:t5, index:id(id, name) range:[1 \"a\",1 \"a\"], [3 \"c\",3 \"c\"], keep order:false, stats:pseudo" + ] + } + ] + } +] diff --git a/pkg/planner/core/casetest/planstats/testdata/plan_stats_suite_out.json b/pkg/planner/core/casetest/planstats/testdata/plan_stats_suite_out.json new file mode 100644 index 0000000000000..e61f53ded7ca2 --- /dev/null +++ b/pkg/planner/core/casetest/planstats/testdata/plan_stats_suite_out.json @@ -0,0 +1,147 @@ +[ + { + "Name": "TestCollectDependingVirtualCols", + "Cases": [ + { + "TableName": "t", + "InputColNames": [ + "a", + "b" + ], + "OutputColNames": [] + }, + { + "TableName": "t", + "InputColNames": [ + "c" + ], + "OutputColNames": [ + "_v$_ic_char_0", + "_v$_ic_signed_0", + "_v$_ic_unsigned_0" + ] + }, + { + "TableName": "t", + "InputColNames": [ + "b", + "c" + ], + "OutputColNames": [ + "_v$_ic_char_0", + "_v$_ic_signed_0", + "_v$_ic_unsigned_0" + ] + }, + { + "TableName": "t1", + "InputColNames": [ + "a" + ], + "OutputColNames": [ + "vab" + ] + }, + { + "TableName": "t1", + "InputColNames": [ + "b" + ], + "OutputColNames": [ + "_v$_ib_0", + "vab", + "vvc" + ] + }, + { + "TableName": "t1", + "InputColNames": [ + "c" + ], + "OutputColNames": [ + "_v$_icvab_0", + "vc" + ] + }, + { + "TableName": "t1", + "InputColNames": [ + "vab" + ], + "OutputColNames": [ + "_v$_icvab_0", + "_v$_ivvcvab_0", + "vvabvvc" + ] + }, + { + "TableName": "t1", + "InputColNames": [ + "vab", + "c" + ], + "OutputColNames": [ + "_v$_icvab_0", + "_v$_ivvcvab_0", + "vc", + "vvabvvc" + ] + }, + { + "TableName": "t1", + "InputColNames": [ + "vc", + "c", + "vvc" + ], + "OutputColNames": [ + "_v$_icvab_0", + "_v$_ivvcvab_0", + "vvabvvc" + ] + } + ] + }, + { + "Name": "TestPartialStatsInExplain", + "Cases": [ + { + "Query": "explain format = brief select * from tp where b = 10", + "Result": [ + "TableReader 0.01 root partition:all data:Selection", + "└─Selection 0.01 cop[tikv] eq(test.tp.b, 10)", + " └─TableFullScan 6.00 cop[tikv] table:tp keep order:false, stats:partial[b:allEvicted]" + ] + }, + { + "Query": "explain format = brief select * from t join tp where tp.a = 10 and t.b = tp.c", + "Result": [ + "Projection 1.00 root test.t.a, test.t.b, test.t.c, test.tp.a, test.tp.b, test.tp.c", + "└─HashJoin 1.00 root inner join, equal:[eq(test.tp.c, test.t.b)]", + " ├─TableReader(Build) 1.00 root partition:p1 data:Selection", + " │ └─Selection 1.00 cop[tikv] eq(test.tp.a, 10), not(isnull(test.tp.c))", + " │ └─TableFullScan 6.00 cop[tikv] table:tp keep order:false, stats:partial[c:allEvicted]", + " └─TableReader(Probe) 3.00 root data:Selection", + " └─Selection 3.00 cop[tikv] not(isnull(test.t.b))", + " └─TableFullScan 3.00 cop[tikv] table:t keep order:false, stats:partial[idx:allEvicted, a:allEvicted, b:allEvicted]" + ] + }, + { + "Query": "explain format = brief select * from t join tp partition (p0) join t2 where t.a < 10 and t.b = tp.c and t2.a > 10 and t2.a = tp.c", + "Result": [ + "HashJoin 0.33 root inner join, equal:[eq(test.tp.c, test.t2.a)]", + "├─IndexJoin(Build) 0.33 root inner join, inner:IndexLookUp, outer key:test.t.b, inner key:test.tp.c, equal cond:eq(test.t.b, test.tp.c)", + "│ ├─TableReader(Build) 0.33 root data:Selection", + "│ │ └─Selection 0.33 cop[tikv] gt(test.t.b, 10), not(isnull(test.t.b))", + "│ │ └─TableRangeScan 1.00 cop[tikv] table:t range:[-inf,10), keep order:false, stats:partial[idx:allEvicted, a:allEvicted, b:allEvicted]", + "│ └─IndexLookUp(Probe) 0.33 root partition:p0 ", + "│ ├─Selection(Build) 0.33 cop[tikv] gt(test.tp.c, 10), not(isnull(test.tp.c))", + "│ │ └─IndexRangeScan 0.50 cop[tikv] table:tp, index:ic(c) range: decided by [eq(test.tp.c, test.t.b)], keep order:false, stats:partial[c:allEvicted]", + "│ └─TableRowIDScan(Probe) 0.33 cop[tikv] table:tp keep order:false, stats:partial[c:allEvicted]", + "└─TableReader(Probe) 1.00 root data:TableRangeScan", + " └─TableRangeScan 1.00 cop[tikv] table:t2 range:(10,+inf], keep order:false, stats:partial[a:allEvicted]" + ] + } + ] + } +] diff --git a/pkg/planner/core/casetest/testdata/integration_suite_out.json b/pkg/planner/core/casetest/testdata/integration_suite_out.json new file mode 100644 index 0000000000000..ffe733cb7fec8 --- /dev/null +++ b/pkg/planner/core/casetest/testdata/integration_suite_out.json @@ -0,0 +1,1388 @@ +[ + { + "Name": "TestIssue31240", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from t31240;", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(test.t31240._tidb_rowid)->Column#6", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t31240 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "set @@tidb_isolation_read_engines=\"tiflash,tidb\";", + "Plan": null + }, + { + "SQL": "explain format = 'brief' select count(*) from t31240;", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(test.t31240._tidb_rowid)->Column#6", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t31240 keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestVerboseExplain", + "Cases": [ + { + "SQL": "explain format = 'verbose' select count(*) from t3", + "Plan": [ + "StreamAgg_20 1.00 102.69 root funcs:count(Column#9)->Column#4", + "└─IndexReader_21 1.00 52.79 root index:StreamAgg_8", + " └─StreamAgg_8 1.00 760.20 cop[tikv] funcs:count(1)->Column#9", + " └─IndexFullScan_19 3.00 610.50 cop[tikv] table:t3, index:c(b) keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t2", + "Plan": [ + "StreamAgg_27 1.00 107.45 root funcs:count(Column#7)->Column#4", + "└─TableReader_28 1.00 57.55 root data:StreamAgg_10", + " └─StreamAgg_10 1.00 831.62 cop[tikv] funcs:count(1)->Column#7", + " └─TableFullScan_25 3.00 681.92 cop[tikv] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select * from t3 order by a", + "Plan": [ + "Sort_4 3.00 318.27 root test.t3.a", + "└─TableReader_8 3.00 70.81 root data:TableFullScan_7", + " └─TableFullScan_7 3.00 681.92 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select * from t3 order by b", + "Plan": [ + "Sort_4 3.00 318.27 root test.t3.b", + "└─TableReader_8 3.00 70.81 root data:TableFullScan_7", + " └─TableFullScan_7 3.00 681.92 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select * from t3 order by a limit 1", + "Plan": [ + "TopN_7 1.00 53.10 root test.t3.a, offset:0, count:1", + "└─TableReader_16 1.00 49.90 root data:TopN_15", + " └─TopN_15 1.00 685.12 cop[tikv] test.t3.a, offset:0, count:1", + " └─TableFullScan_14 3.00 681.92 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select * from t3 order by b limit 1", + "Plan": [ + "TopN_7 1.00 53.10 root test.t3.b, offset:0, count:1", + "└─TableReader_16 1.00 49.90 root data:TopN_15", + " └─TopN_15 1.00 685.12 cop[tikv] test.t3.b, offset:0, count:1", + " └─TableFullScan_14 3.00 681.92 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t2 group by a", + "Plan": [ + "HashAgg_8 3.00 1706.09 root group by:test.t2.a, funcs:count(1)->Column#4", + "└─TableReader_17 3.00 58.13 root data:TableFullScan_16", + " └─TableFullScan_16 3.00 681.92 cop[tikv] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t3 where b = 0", + "Plan": [ + "StreamAgg_10 1.00 64.98 root funcs:count(1)->Column#4", + "└─IndexReader_15 1.00 15.08 root index:IndexRangeScan_14", + " └─IndexRangeScan_14 1.00 162.80 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select /*+ use_index(t3, c) */ count(a) from t3 where b = 0", + "Plan": [ + "StreamAgg_10 1.00 2001.63 root funcs:count(test.t3.a)->Column#4", + "└─IndexLookUp_17 1.00 1951.73 root ", + " ├─IndexRangeScan_15(Build) 1.00 203.50 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false", + " └─TableRowIDScan_16(Probe) 1.00 227.31 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t2 where a = 0", + "Plan": [ + "StreamAgg_12 1.00 109.57 root funcs:count(1)->Column#4", + "└─TableReader_20 1.00 59.67 root data:Selection_19", + " └─Selection_19 1.00 831.62 cop[tikv] eq(test.t2.a, 0)", + " └─TableFullScan_18 3.00 681.92 cop[tikv] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t3 t join t3 on t.a = t3.b", + "Plan": [ + "StreamAgg_10 1.00 2128.93 root funcs:count(1)->Column#7", + "└─HashJoin_40 3.00 1979.23 root inner join, equal:[eq(test.t3.a, test.t3.b)]", + " ├─IndexReader_28(Build) 3.00 45.23 root index:IndexFullScan_27", + " │ └─IndexFullScan_27 3.00 488.40 cop[tikv] table:t3, index:c(b) keep order:false", + " └─TableReader_26(Probe) 3.00 68.11 root data:Selection_25", + " └─Selection_25 3.00 831.62 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan_24 3.00 681.92 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", + "Plan": [ + "StreamAgg_15 1.00 62049.70 root funcs:count(1)->Column#7", + "└─TableReader_46 3.00 61900.00 root MppVersion: 2, data:ExchangeSender_45", + " └─ExchangeSender_45 3.00 928447.21 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_44 3.00 928447.21 mpp[tiflash] test.t1.a", + " └─HashJoin_41 3.00 928447.20 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_22(Build) 3.00 464290.40 mpp[tiflash] ", + " │ └─ExchangeSender_21 3.00 464146.40 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_20 3.00 464146.40 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan_19 3.00 464139.20 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false", + " └─Selection_24(Probe) 3.00 464146.40 mpp[tiflash] not(isnull(test.t2.a))", + " └─TableFullScan_23 3.00 464139.20 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", + "Plan": [ + "StreamAgg_15 1.00 71701.80 root funcs:count(1)->Column#10", + "└─HashJoin_63 3.00 71552.10 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─TableReader_40(Build) 3.00 69645.79 root MppVersion: 2, data:ExchangeSender_39", + " │ └─ExchangeSender_39 3.00 1044634.01 mpp[tiflash] ExchangeType: PassThrough", + " │ └─Projection_38 3.00 1044634.01 mpp[tiflash] test.t1.b", + " │ └─HashJoin_29 3.00 1044634.00 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─ExchangeReceiver_35(Build) 3.00 580476.40 mpp[tiflash] ", + " │ │ └─ExchangeSender_34 3.00 580188.40 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection_33 3.00 580188.40 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ │ └─TableFullScan_32 3.00 580174.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false", + " │ └─Selection_37(Probe) 3.00 464146.40 mpp[tiflash] not(isnull(test.t2.a))", + " │ └─TableFullScan_36 3.00 464139.20 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false", + " └─IndexReader_50(Probe) 3.00 45.23 root index:IndexFullScan_49", + " └─IndexFullScan_49 3.00 488.40 cop[tikv] table:t3, index:c(b) keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select (2) in (select /*+ read_from_storage(tiflash[t1]) */ count(*) from t1) from (select t.b < (select /*+ read_from_storage(tiflash[t2]) */ t.b from t2 limit 1 ) from t3 t) t; -- we do generate the agg pushed-down plan of mpp, but cost-cmp failed", + "Plan": [ + "HashJoin_17 3.00 32770.77 root CARTESIAN left outer semi join", + "├─Selection_22(Build) 0.80 31149.25 root eq(2, Column#18)", + "│ └─StreamAgg_29 1.00 31099.35 root funcs:count(1)->Column#18", + "│ └─TableReader_41 3.00 30949.65 root MppVersion: 2, data:ExchangeSender_40", + "│ └─ExchangeSender_40 3.00 464139.20 mpp[tiflash] ExchangeType: PassThrough", + "│ └─TableFullScan_39 3.00 464139.20 mpp[tiflash] table:t1 keep order:false", + "└─IndexReader_21(Probe) 3.00 53.37 root index:IndexFullScan_20", + " └─IndexFullScan_20 3.00 610.50 cop[tikv] table:t, index:c(b) keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select /*+ merge_join(t1), read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", + "Plan": [ + "StreamAgg_15 1.00 62546.70 root funcs:count(1)->Column#7", + "└─MergeJoin_29 3.00 62397.00 root inner join, left key:test.t1.a, right key:test.t2.a", + " ├─Sort_27(Build) 3.00 31197.00 root test.t2.a", + " │ └─TableReader_26 3.00 30950.13 root MppVersion: 2, data:ExchangeSender_25", + " │ └─ExchangeSender_25 3.00 464146.40 mpp[tiflash] ExchangeType: PassThrough", + " │ └─Selection_24 3.00 464146.40 mpp[tiflash] not(isnull(test.t2.a))", + " │ └─TableFullScan_23 3.00 464139.20 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false", + " └─Sort_22(Probe) 3.00 31197.00 root test.t1.a", + " └─TableReader_21 3.00 30950.13 root MppVersion: 2, data:ExchangeSender_20", + " └─ExchangeSender_20 3.00 464146.40 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection_19 3.00 464146.40 mpp[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan_18 3.00 464139.20 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false" + ] + } + ] + }, + { + "Name": "TestIsolationReadDoNotFilterSystemDB", + "Cases": [ + { + "SQL": "desc format = 'brief' select * from metrics_schema.tidb_query_duration where time >= '2019-12-23 16:10:13' and time <= '2019-12-23 16:30:13'", + "Plan": [ + "MemTableScan 10000.00 root table:tidb_query_duration PromQL:histogram_quantile(0.9, sum(rate(tidb_server_handle_query_duration_seconds_bucket{}[60s])) by (le,sql_type,instance)), start_time:2019-12-23 16:10:13, end_time:2019-12-23 16:30:13, step:1m0s" + ] + }, + { + "SQL": "desc format = 'brief' select * from information_schema.tables", + "Plan": [ + "MemTableScan 10000.00 root table:TABLES " + ] + }, + { + "SQL": "desc format = 'brief' select * from mysql.stats_meta", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:stats_meta keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIsolationReadTiFlashNotChoosePointGet", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t where t.a = 1", + "Result": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableRangeScan 1.00 mpp[tiflash] table:t range:[1,1], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where t.a in (1, 2)", + "Result": [ + "TableReader 2.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 2.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableRangeScan 2.00 mpp[tiflash] table:t range:[1,1], [2,2], keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestMergeContinuousSelections", + "Cases": [ + { + "SQL": "desc format = 'brief' SELECT table2 . `col_char_64` AS field1 FROM `ts` AS table2 INNER JOIN (SELECT DISTINCT SUBQUERY3_t1 . * FROM `ts` AS SUBQUERY3_t1 LEFT OUTER JOIN `ts` AS SUBQUERY3_t2 ON SUBQUERY3_t2 . `col_varchar_64_not_null` = SUBQUERY3_t1 . `col_varchar_key`) AS table3 ON (table3 . `col_varchar_key` = table2 . `col_varchar_64`) WHERE table3 . `col_char_64_not_null` >= SOME (SELECT SUBQUERY4_t1 . `col_varchar_64` AS SUBQUERY4_field1 FROM `ts` AS SUBQUERY4_t1) GROUP BY field1 ;", + "Plan": [ + "HashAgg 7992.00 root group by:test.ts.col_char_64, funcs:firstrow(test.ts.col_char_64)->test.ts.col_char_64", + "└─HashJoin 9990.00 root CARTESIAN inner join, other cond:or(ge(test.ts.col_char_64_not_null, Column#25), if(ne(Column#26, 0), NULL, 0))", + " ├─Selection(Build) 0.80 root ne(Column#27, 0)", + " │ └─HashAgg 1.00 root funcs:min(Column#36)->Column#25, funcs:sum(Column#37)->Column#26, funcs:count(Column#38)->Column#27", + " │ └─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " │ └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 mpp[tiflash] funcs:min(Column#42)->Column#36, funcs:sum(Column#43)->Column#37, funcs:count(1)->Column#38", + " │ └─Projection 10000.00 mpp[tiflash] test.ts.col_varchar_64->Column#42, cast(isnull(test.ts.col_varchar_64), decimal(20,0) BINARY)->Column#43", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:SUBQUERY4_t1 keep order:false, stats:pseudo", + " └─TableReader(Probe) 12487.50 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 12487.50 mpp[tiflash] test.ts.col_char_64, test.ts.col_char_64_not_null", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.ts.col_varchar_64, test.ts.col_varchar_key)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.ts.col_varchar_64))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:table2 pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.ts.col_varchar_key))", + " └─TableFullScan 10000.00 mpp[tiflash] table:SUBQUERY3_t1 pushed down filter:empty, keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestPushDownGroupConcatToTiFlash", + "Cases": [ + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts", + "Plan": [ + "HashAgg 1.00 root funcs:group_concat(Column#6 separator \",\")->Column#5", + "└─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#8, Column#9, Column#10 separator \",\")->Column#6", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#8, test.ts.col_1->Column#9, cast(test.ts.id, var_string(20))->Column#10", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#6, Column#7, Column#8 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0->Column#6, test.ts.col_1->Column#7, cast(test.ts.id, var_string(20))->Column#8", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#6, test.ts.col_1->Column#7, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0->Column#9", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0->Column#6, test.ts.col_1->Column#7, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0->Column#9", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(*),min(col_1) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#8, Column#9, Column#10 order by Column#11 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:min(Column#12)->Column#7", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#8, test.ts.col_1->Column#9, cast(test.ts.id, var_string(20))->Column#10, test.ts.col_0->Column#11, test.ts.col_1->Column#12", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(*),max(col_0) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#12, Column#13, Column#14 order by Column#15 separator \",\")->Column#5, funcs:sum(Column#16)->Column#6, funcs:max(Column#17)->Column#7", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0->Column#12, test.ts.col_1->Column#13, cast(test.ts.id, var_string(20))->Column#14, test.ts.col_0->Column#15, Column#10->Column#16, Column#11->Column#17", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, funcs:count(1)->Column#10, funcs:max(test.ts.col_0)->Column#11", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#12, funcs:group_concat(Column#9, Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#9, test.ts.col_1->Column#10, cast(test.ts.id, var_string(20))->Column#11, test.ts.col_2->Column#12", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#9, funcs:group_concat(distinct Column#6, Column#7, Column#8 separator \",\")->Column#5", + " └─Projection 8000.00 mpp[tiflash] test.ts.col_0->Column#6, test.ts.col_1->Column#7, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_2->Column#9", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.col_2, test.ts.id, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, funcs:group_concat(Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#6, test.ts.col_1->Column#7, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0->Column#9, test.ts.col_2->Column#10", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, funcs:group_concat(distinct Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 8000.00 mpp[tiflash] test.ts.col_0->Column#6, test.ts.col_1->Column#7, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0->Column#9, test.ts.col_2->Column#10", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.col_2, test.ts.id, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_1, id order by col_0) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#9, funcs:group_concat(Column#6, Column#7 order by Column#8 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_1->Column#6, cast(test.ts.id, var_string(20))->Column#7, test.ts.col_0->Column#8, test.ts.col_2->Column#9", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_1, id order by col_0) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#9, funcs:group_concat(distinct Column#6, Column#7 order by Column#8 separator \",\")->Column#5", + " └─Projection 8000.00 mpp[tiflash] test.ts.col_1->Column#6, cast(test.ts.id, var_string(20))->Column#7, test.ts.col_0->Column#8, test.ts.col_2->Column#9", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_1, test.ts.col_2, test.ts.id, funcs:firstrow(test.ts.col_0)->test.ts.col_0", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(*),min(col_0),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#20, funcs:group_concat(Column#13, Column#14, Column#15 order by Column#16 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:min(Column#17)->Column#7, funcs:count(Column#18)->Column#11, funcs:sum(Column#19)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#13, test.ts.col_1->Column#14, cast(test.ts.id, var_string(20))->Column#15, test.ts.col_0->Column#16, test.ts.col_0->Column#17, test.ts.id->Column#18, cast(test.ts.id, decimal(10,0) BINARY)->Column#19, test.ts.col_2->Column#20", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(*),max(col_1),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#19, 0), 1, Column#19), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#32, funcs:group_concat(distinct Column#25, Column#26, Column#27 order by Column#28 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:max(Column#29)->Column#7, funcs:count(Column#30)->Column#19, funcs:sum(Column#31)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#25, test.ts.col_1->Column#26, cast(test.ts.id, var_string(20))->Column#27, test.ts.col_0->Column#28, test.ts.col_1->Column#29, test.ts.id->Column#30, cast(test.ts.id, decimal(10,0) BINARY)->Column#31, test.ts.col_2->Column#32", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(distinct id),min(col_0),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#10, 0), 1, Column#10), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#11, Column#12, Column#13 order by Column#14 separator \",\")->Column#5, funcs:count(Column#15)->Column#6, funcs:min(Column#16)->Column#7, funcs:count(Column#17)->Column#10, funcs:sum(Column#18)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#11, test.ts.col_1->Column#12, cast(test.ts.id, var_string(20))->Column#13, test.ts.col_0->Column#14, test.ts.id->Column#15, test.ts.col_0->Column#16, test.ts.id->Column#17, cast(test.ts.id, decimal(10,0) BINARY)->Column#18", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(distinct id),max(col_1),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#26, Column#27, Column#28 order by Column#29 separator \",\")->Column#5, funcs:sum(Column#30)->Column#6, funcs:max(Column#31)->Column#7, funcs:sum(Column#32)->Column#14, funcs:sum(Column#33)->Column#8", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0->Column#26, test.ts.col_1->Column#27, cast(test.ts.id, var_string(20))->Column#28, test.ts.col_0->Column#29, Column#15->Column#30, Column#16->Column#31, Column#17->Column#32, Column#18->Column#33", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#23, Column#24, Column#25, funcs:count(Column#19)->Column#15, funcs:max(Column#20)->Column#16, funcs:count(Column#21)->Column#17, funcs:sum(Column#22)->Column#18", + " └─Projection 10000.00 mpp[tiflash] test.ts.id->Column#19, test.ts.col_1->Column#20, test.ts.id->Column#21, cast(test.ts.id, decimal(10,0) BINARY)->Column#22, test.ts.col_0->Column#23, test.ts.col_1->Column#24, test.ts.id->Column#25", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),min(col_0),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#21, 0), 1, Column#21), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#35, funcs:group_concat(Column#28, Column#29, Column#30 separator \",\")->Column#5, funcs:count(Column#31)->Column#6, funcs:min(Column#32)->Column#7, funcs:count(Column#33)->Column#21, funcs:sum(Column#34)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#28, test.ts.col_1->Column#29, cast(test.ts.id, var_string(20))->Column#30, test.ts.id->Column#31, test.ts.col_0->Column#32, test.ts.id->Column#33, cast(test.ts.id, decimal(10,0) BINARY)->Column#34, test.ts.col_2->Column#35", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),max(col_1),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#19, 0), 1, Column#19), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#32, funcs:group_concat(distinct Column#25, Column#26, Column#27 separator \",\")->Column#5, funcs:count(Column#28)->Column#6, funcs:max(Column#29)->Column#7, funcs:count(Column#30)->Column#19, funcs:sum(Column#31)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#25, test.ts.col_1->Column#26, cast(test.ts.id, var_string(20))->Column#27, test.ts.id->Column#28, test.ts.col_1->Column#29, test.ts.id->Column#30, cast(test.ts.id, decimal(10,0) BINARY)->Column#31, test.ts.col_2->Column#32", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),min(col_0),avg(id) from ts", + "Plan": [ + "HashAgg 1.00 root funcs:group_concat(Column#9 separator \",\")->Column#5, funcs:count(Column#10)->Column#6, funcs:min(Column#11)->Column#7, funcs:avg(Column#12, Column#13)->Column#8", + "└─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#19, Column#20, Column#21 separator \",\")->Column#9, funcs:count(Column#22)->Column#10, funcs:min(Column#23)->Column#11, funcs:count(Column#24)->Column#12, funcs:sum(Column#25)->Column#13", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#19, test.ts.col_1->Column#20, cast(test.ts.id, var_string(20))->Column#21, test.ts.id->Column#22, test.ts.col_0->Column#23, test.ts.id->Column#24, cast(test.ts.id, decimal(10,0) BINARY)->Column#25", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),max(col_1),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#26, Column#27, Column#28 separator \",\")->Column#5, funcs:sum(Column#29)->Column#6, funcs:max(Column#30)->Column#7, funcs:sum(Column#31)->Column#14, funcs:sum(Column#32)->Column#8", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0->Column#26, test.ts.col_1->Column#27, cast(test.ts.id, var_string(20))->Column#28, Column#15->Column#29, Column#16->Column#30, Column#17->Column#31, Column#18->Column#32", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#23, Column#24, Column#25, funcs:count(Column#19)->Column#15, funcs:max(Column#20)->Column#16, funcs:count(Column#21)->Column#17, funcs:sum(Column#22)->Column#18", + " └─Projection 10000.00 mpp[tiflash] test.ts.id->Column#19, test.ts.col_1->Column#20, test.ts.id->Column#21, cast(test.ts.id, decimal(10,0) BINARY)->Column#22, test.ts.col_0->Column#23, test.ts.col_1->Column#24, test.ts.id->Column#25", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),group_concat(col_0 order by 1),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#15, 0), 1, Column#15), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#27, funcs:group_concat(Column#19, Column#20, Column#21 separator \",\")->Column#5, funcs:count(Column#22)->Column#6, funcs:group_concat(Column#23 order by Column#24 separator \",\")->Column#7, funcs:count(Column#25)->Column#15, funcs:sum(Column#26)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#19, test.ts.col_1->Column#20, cast(test.ts.id, var_string(20))->Column#21, test.ts.id->Column#22, test.ts.col_0->Column#23, test.ts.col_0->Column#24, test.ts.id->Column#25, cast(test.ts.id, decimal(10,0) BINARY)->Column#26, test.ts.col_2->Column#27", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0),count(distinct id),group_concat(col_1, id order by 1,2),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#13, 0), 1, Column#13), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#24, funcs:group_concat(distinct Column#16 separator \",\")->Column#5, funcs:count(Column#17)->Column#6, funcs:group_concat(Column#18, Column#19 order by Column#20, Column#21 separator \",\")->Column#7, funcs:count(Column#22)->Column#13, funcs:sum(Column#23)->Column#8", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#16, test.ts.id->Column#17, test.ts.col_1->Column#18, cast(test.ts.id, var_string(20))->Column#19, test.ts.col_1->Column#20, test.ts.id->Column#21, test.ts.id->Column#22, cast(test.ts.id, decimal(10,0) BINARY)->Column#23, test.ts.col_2->Column#24", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, id),count(distinct id),group_concat(col_1, id order by 1,2),min(col_0),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#13, 0), 1, Column#13), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(Column#16, Column#17 separator \",\")->Column#5, funcs:count(Column#18)->Column#6, funcs:group_concat(Column#19, Column#20 order by Column#21, Column#22 separator \",\")->Column#7, funcs:min(Column#23)->Column#8, funcs:count(Column#24)->Column#13, funcs:sum(Column#25)->Column#9", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#16, cast(test.ts.id, var_string(20))->Column#17, test.ts.id->Column#18, test.ts.col_1->Column#19, cast(test.ts.id, var_string(20))->Column#20, test.ts.col_1->Column#21, test.ts.id->Column#22, test.ts.col_0->Column#23, test.ts.id->Column#24, cast(test.ts.id, decimal(10,0) BINARY)->Column#25", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),group_concat(col_1, id order by 1,2),max(col_1),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#12, 0), 1, Column#12), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#14, Column#15, Column#16 separator \",\")->Column#5, funcs:count(Column#17)->Column#6, funcs:group_concat(Column#18, Column#19 order by Column#20, Column#21 separator \",\")->Column#7, funcs:max(Column#22)->Column#8, funcs:count(Column#23)->Column#12, funcs:sum(Column#24)->Column#9", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#14, test.ts.col_1->Column#15, cast(test.ts.id, var_string(20))->Column#16, test.ts.id->Column#17, test.ts.col_1->Column#18, cast(test.ts.id, var_string(20))->Column#19, test.ts.col_1->Column#20, test.ts.id->Column#21, test.ts.col_1->Column#22, test.ts.id->Column#23, cast(test.ts.id, decimal(10,0) BINARY)->Column#24", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct col_2),group_concat(col_1, id),max(col_1),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#15, 0), 1, Column#15), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#29, Column#30, Column#31 separator \",\")->Column#5, funcs:count(distinct Column#32)->Column#6, funcs:group_concat(Column#33 separator \",\")->Column#7, funcs:max(Column#34)->Column#8, funcs:sum(Column#35)->Column#15, funcs:sum(Column#36)->Column#9", + " └─Projection 1.00 mpp[tiflash] test.ts.col_0->Column#29, test.ts.col_1->Column#30, cast(test.ts.id, var_string(20))->Column#31, test.ts.col_2->Column#32, Column#16->Column#33, Column#17->Column#34, Column#18->Column#35, Column#19->Column#36", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#25, Column#26, Column#27, Column#28, funcs:group_concat(Column#20, Column#21 separator \",\")->Column#16, funcs:max(Column#22)->Column#17, funcs:count(Column#23)->Column#18, funcs:sum(Column#24)->Column#19", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_1->Column#20, cast(test.ts.id, var_string(20))->Column#21, test.ts.col_1->Column#22, test.ts.id->Column#23, cast(test.ts.id, decimal(10,0) BINARY)->Column#24, test.ts.col_0->Column#25, test.ts.col_1->Column#26, test.ts.id->Column#27, test.ts.col_2->Column#28", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct col_2),group_concat(col_1, id),max(col_1),avg(id) from ts group by col_0", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#20, 0), 1, Column#20), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#35, funcs:group_concat(distinct Column#26, Column#27, Column#28 separator \",\")->Column#5, funcs:count(distinct Column#29)->Column#6, funcs:group_concat(Column#30, Column#31 separator \",\")->Column#7, funcs:max(Column#32)->Column#8, funcs:count(Column#33)->Column#20, funcs:sum(Column#34)->Column#9", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#26, test.ts.col_1->Column#27, cast(test.ts.id, var_string(20))->Column#28, test.ts.col_2->Column#29, test.ts.col_1->Column#30, cast(test.ts.id, var_string(20))->Column#31, test.ts.col_1->Column#32, test.ts.id->Column#33, cast(test.ts.id, decimal(10,0) BINARY)->Column#34, test.ts.col_0->Column#35", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_0, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#8, var_string(20))->Column#10, Column#9->Column#11", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", 0, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'01') from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#8, var_string(20))->Column#10, Column#9->Column#11", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:\"01\", 0, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,1) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#8, var_string(20))->Column#10, cast(Column#9, var_string(20))->Column#11", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:0, 1, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#8, Column#9 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#7, var_string(20))->Column#8, cast(Column#7, var_string(20))->Column#9", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:0, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,10) from ts group by '010'", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#17, funcs:group_concat(distinct Column#15, Column#16 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#13, var_string(20))->Column#15, cast(Column#14, var_string(20))->Column#16, Column#12->Column#17", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#12, collate: binary]", + " └─HashAgg 1.00 mpp[tiflash] group by:0, 1, 10, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts group by '011'", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#14, funcs:group_concat(distinct Column#12, Column#13 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#11, var_string(20))->Column#12, cast(Column#11, var_string(20))->Column#13, Column#10->Column#14", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#10, collate: binary]", + " └─HashAgg 1.00 mpp[tiflash] group by:0, 1, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts group by 'GG'", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#17, funcs:group_concat(distinct Column#15, Column#16 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] cast(Column#13, var_string(20))->Column#15, Column#14->Column#16, Column#12->Column#17", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#12, collate: binary]", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", 0, 1, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG','GG') from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#7, Column#7 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'Gg','GG') from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#8, Column#9 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", \"Gg\", ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG-10','GG') from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#8, Column#9 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:\"GG\", \"GG-10\", ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct '1200-01-01 00:00:00.023',1200) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 1.00 mpp[tiflash] Column#8->Column#10, cast(Column#9, var_string(20))->Column#11", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:\"1200-01-01 00:00:00.023\", 1200, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", + "[types:1292]Truncated incorrect DOUBLE value: '1200-01-01 00:00:00.023'", + "[types:1292]Truncated incorrect DOUBLE value: '1200-01-01 00:00:00.023'" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0) from ts group by id", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.id, funcs:group_concat(test.ts.col_0, test.ts.col_0 separator \",\")->Column#5", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.id, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0,id) from ts group by id", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#12, funcs:group_concat(Column#9, Column#10, Column#11 separator \",\")->Column#5", + " └─Projection 10000.00 mpp[tiflash] test.ts.col_0->Column#9, test.ts.col_0->Column#10, cast(test.ts.id, var_string(20))->Column#11, test.ts.id->Column#12", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.id, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts", + "Plan": [ + "TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#5", + " └─HashAgg 1.00 mpp[tiflash] funcs:group_concat(distinct test.ts.col_0 order by Column#7 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─HashAgg 1.00 mpp[tiflash] group by:Column#9, funcs:firstrow(Column#8)->Column#7", + " └─Projection 10000.00 mpp[tiflash] lt(test.ts.id, 10)->Column#8, test.ts.col_0->Column#9", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts group by col_1", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.ts.col_1, funcs:group_concat(distinct test.ts.col_0 order by Column#8 separator \",\")->Column#5", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_1, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, Column#11, funcs:firstrow(Column#9)->Column#8", + " └─Projection 10000.00 mpp[tiflash] lt(test.ts.id, 10)->Column#9, test.ts.col_1->Column#10, test.ts.col_0->Column#11", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0>10 order by id<10) from ts group by col_1", + "Plan": [ + "TableReader 8000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] Column#5", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#17, funcs:group_concat(distinct Column#15 order by Column#16 separator \",\")->Column#5", + " └─Projection 8000.00 mpp[tiflash] cast(Column#10, var_string(20))->Column#15, Column#11->Column#16, test.ts.col_1->Column#17", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.ts.col_1, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, Column#14, funcs:firstrow(Column#12)->Column#11", + " └─Projection 10000.00 mpp[tiflash] lt(test.ts.id, 10)->Column#12, test.ts.col_1->Column#13, gt(cast(test.ts.col_0, double BINARY), 10)->Column#14", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by col_0<=>null) from ts", + "Plan": [ + "HashAgg 1.00 root funcs:group_concat(distinct Column#6 order by Column#7 separator \",\")->Column#5", + "└─Projection 10000.00 root test.ts.col_0->Column#6, nulleq(test.ts.col_0, )->Column#7", + " └─TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", + "Scalar function 'nulleq'(signature: NullEQString, return type: bigint(1)) is not supported to push down to tiflash now.", + "Aggregation can not be pushed to tiflash because arguments of AggFunc `group_concat` contains unsupported exprs in order-by clause", + "Scalar function 'nulleq'(signature: NullEQString, return type: bigint(1)) is not supported to push down to tiflash now.", + "Aggregation can not be pushed to tiflash because arguments of AggFunc `group_concat` contains unsupported exprs in order-by clause" + ] + } + ] + }, + { + "Name": "TestIssue32632", + "Cases": [ + { + "SQL": "explain format = 'brief' select sum(ps_supplycost) from partsupp, supplier where ps_suppkey = s_suppkey;", + "Plan": [ + "HashAgg 1.00 root funcs:sum(Column#15)->Column#14", + "└─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:sum(test.partsupp.ps_supplycost)->Column#15", + " └─Projection 12500.00 mpp[tiflash] test.partsupp.ps_supplycost", + " └─HashJoin 12500.00 mpp[tiflash] inner join, equal:[eq(test.supplier.s_suppkey, test.partsupp.ps_suppkey)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:supplier keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 800000.00 mpp[tiflash] table:partsupp keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestTiFlashPartitionTableScan", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from rp_t where a = 1 or a = 20", + "Plan": [ + "TableReader 20.00 root partition:p0,p3 MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 20.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 20.00 mpp[tiflash] or(eq(test.rp_t.a, 1), eq(test.rp_t.a, 20))", + " └─TableFullScan 10000.00 mpp[tiflash] table:rp_t pushed down filter:empty, keep order:false, stats:pseudo, PartitionTableScan:true" + ] + }, + { + "SQL": "explain format = 'brief' select * from hp_t where a = 1 or a = 20", + "Plan": [ + "TableReader 20.00 root partition:p0,p1 MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 20.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 20.00 mpp[tiflash] or(eq(test.hp_t.a, 1), eq(test.hp_t.a, 20))", + " └─TableFullScan 10000.00 mpp[tiflash] table:hp_t pushed down filter:empty, keep order:false, stats:pseudo, PartitionTableScan:true" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from rp_t where a = 1 or a = 20", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#4)->Column#3", + "└─TableReader 1.00 root partition:p0,p3 MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#4", + " └─Selection 20.00 mpp[tiflash] or(eq(test.rp_t.a, 1), eq(test.rp_t.a, 20))", + " └─TableFullScan 10000.00 mpp[tiflash] table:rp_t pushed down filter:empty, keep order:false, stats:pseudo, PartitionTableScan:true" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from hp_t where a = 1 or a = 20", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#4)->Column#3", + "└─TableReader 1.00 root partition:p0,p1 MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#4", + " └─Selection 20.00 mpp[tiflash] or(eq(test.hp_t.a, 1), eq(test.hp_t.a, 20))", + " └─TableFullScan 10000.00 mpp[tiflash] table:hp_t pushed down filter:empty, keep order:false, stats:pseudo, PartitionTableScan:true" + ] + } + ] + }, + { + "Name": "TestTiFlashFineGrainedShuffle", + "Cases": [ + { + "SQL": "explain format = 'brief' select row_number() over w1 from t1 window w1 as (partition by c1 order by c1);", + "Plan": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] Column#5->Column#6, stream_count: 8", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#5 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] Column#5->Column#6, stream_count: 8", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#5 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1, rank() over w2 from t1 window w1 as (partition by c1 order by c1), w2 as (partition by c2);", + "Plan": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] Column#7->Column#8, Column#6->Column#9, stream_count: 8", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#7 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Window 10000.00 mpp[tiflash] rank()->Column#6 over(partition by test.t1.c2), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] Column#7->Column#8, Column#6->Column#9, stream_count: 8", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#7 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Window 10000.00 mpp[tiflash] rank()->Column#6 over(partition by test.t1.c2), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1, rank() over w2 from t1 window w1 as (partition by c1 order by c1), w2 as (partition by c2) order by 1, 2 limit 10;", + "Plan": [ + "Projection 10.00 root Column#7->Column#8, Column#6->Column#9", + "└─TopN 10.00 root Column#7, Column#6, offset:0, count:10", + " └─TableReader 10.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TopN 10.00 mpp[tiflash] Column#7, Column#6, offset:0, count:10", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#7 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Window 10000.00 mpp[tiflash] rank()->Column#6 over(partition by test.t1.c2), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Redact": [ + "Projection 10.00 root Column#7->Column#8, Column#6->Column#9", + "└─TopN 10.00 root Column#7, Column#6, offset:?, count:?", + " └─TableReader 10.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TopN 10.00 mpp[tiflash] Column#7, Column#6, offset:?, count:?", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#7 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Window 10000.00 mpp[tiflash] rank()->Column#6 over(partition by test.t1.c2), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1, count(c2) from t1 group by c1 having c1 > 10 window w1 as (partition by c2 order by c2);", + "Plan": [ + "TableReader 2666.67 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2666.67 mpp[tiflash] Column#6->Column#7, Column#4->Column#8, stream_count: 8", + " └─Window 2666.67 mpp[tiflash] row_number()->Column#6 over(partition by test.t1.c2 order by test.t1.c2 rows between current row and current row), stream_count: 8", + " └─Sort 2666.67 mpp[tiflash] test.t1.c2, test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 2666.67 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", + " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c2", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:sum(Column#9)->Column#4, funcs:firstrow(Column#10)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#9, funcs:firstrow(test.t1.c2)->Column#10", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, 10)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 2666.67 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2666.67 mpp[tiflash] Column#6->Column#7, Column#4->Column#8, stream_count: 8", + " └─Window 2666.67 mpp[tiflash] row_number()->Column#6 over(partition by test.t1.c2 order by test.t1.c2 rows between current row and current row), stream_count: 8", + " └─Sort 2666.67 mpp[tiflash] test.t1.c2, test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 2666.67 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", + " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c2", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:sum(Column#9)->Column#4, funcs:firstrow(Column#10)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#9, funcs:firstrow(test.t1.c2)->Column#10", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, ?)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1, count(c1) from t1 group by c2 having c2 > 10 window w1 as (partition by c1 order by c2);", + "Plan": [ + "TableReader 2666.67 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2666.67 mpp[tiflash] Column#6->Column#7, Column#4->Column#8, stream_count: 8", + " └─Window 2666.67 mpp[tiflash] row_number()->Column#6 over(partition by test.t1.c1 order by test.t1.c2 rows between current row and current row), stream_count: 8", + " └─Sort 2666.67 mpp[tiflash] test.t1.c1, test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 2666.67 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c1, test.t1.c2", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:sum(Column#9)->Column#4, funcs:firstrow(Column#10)->test.t1.c1, funcs:firstrow(test.t1.c2)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:count(test.t1.c1)->Column#9, funcs:firstrow(test.t1.c1)->Column#10", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c2, 10)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 2666.67 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2666.67 mpp[tiflash] Column#6->Column#7, Column#4->Column#8, stream_count: 8", + " └─Window 2666.67 mpp[tiflash] row_number()->Column#6 over(partition by test.t1.c1 order by test.t1.c2 rows between current row and current row), stream_count: 8", + " └─Sort 2666.67 mpp[tiflash] test.t1.c1, test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 2666.67 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c1, test.t1.c2", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:sum(Column#9)->Column#4, funcs:firstrow(Column#10)->test.t1.c1, funcs:firstrow(test.t1.c2)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:count(test.t1.c1)->Column#9, funcs:firstrow(test.t1.c1)->Column#10", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c2, ?)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1 from t1 a join t1 b on a.c1 = b.c2 window w1 as (partition by a.c1);", + "Plan": [ + "TableReader 12487.50 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 12487.50 mpp[tiflash] Column#8->Column#9, stream_count: 8", + " └─Window 12487.50 mpp[tiflash] row_number()->Column#8 over(partition by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 12487.50 mpp[tiflash] test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 12487.50 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Projection 12487.50 mpp[tiflash] test.t1.c1", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t1.c1, test.t1.c2)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.c1))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:a pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t1.c2))", + " └─TableFullScan 10000.00 mpp[tiflash] table:b pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 12487.50 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 12487.50 mpp[tiflash] Column#8->Column#9, stream_count: 8", + " └─Window 12487.50 mpp[tiflash] row_number()->Column#8 over(partition by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 12487.50 mpp[tiflash] test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 12487.50 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Projection 12487.50 mpp[tiflash] test.t1.c1", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t1.c1, test.t1.c2)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.c1))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:a pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t1.c2))", + " └─TableFullScan 10000.00 mpp[tiflash] table:b pushed down filter:empty, keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1 from t1 where c1 < 100 window w1 as (partition by c1 order by c1);", + "Plan": [ + "TableReader 3323.33 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 3323.33 mpp[tiflash] Column#5->Column#6, stream_count: 8", + " └─Window 3323.33 mpp[tiflash] row_number()->Column#5 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 3323.33 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 3323.33 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Selection 3323.33 mpp[tiflash] lt(test.t1.c1, 100)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 3323.33 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 3323.33 mpp[tiflash] Column#5->Column#6, stream_count: 8", + " └─Window 3323.33 mpp[tiflash] row_number()->Column#5 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 3323.33 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 3323.33 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Selection 3323.33 mpp[tiflash] lt(test.t1.c1, ?)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t1;", + "Plan": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1 from t1 window w1 as (order by c1);", + "Plan": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] Column#5->Column#6", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#5 over(order by test.t1.c1 rows between current row and current row)", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] Column#5->Column#6", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#5 over(order by test.t1.c1 rows between current row and current row)", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select row_number() over w1, count(c2) from t1 group by c1 having c1 > 10 window w1 as (partition by c1 order by c2);", + "Plan": [ + "TableReader 2666.67 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2666.67 mpp[tiflash] Column#6->Column#7, Column#4->Column#8", + " └─Window 2666.67 mpp[tiflash] row_number()->Column#6 over(partition by test.t1.c1 order by test.t1.c2 rows between current row and current row)", + " └─Sort 2666.67 mpp[tiflash] test.t1.c1, test.t1.c2", + " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c1, test.t1.c2", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:sum(Column#9)->Column#4, funcs:firstrow(test.t1.c1)->test.t1.c1, funcs:firstrow(Column#11)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#9, funcs:firstrow(test.t1.c2)->Column#11", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, 10)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 2666.67 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2666.67 mpp[tiflash] Column#6->Column#7, Column#4->Column#8", + " └─Window 2666.67 mpp[tiflash] row_number()->Column#6 over(partition by test.t1.c1 order by test.t1.c2 rows between current row and current row)", + " └─Sort 2666.67 mpp[tiflash] test.t1.c1, test.t1.c2", + " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c1, test.t1.c2", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:sum(Column#9)->Column#4, funcs:firstrow(test.t1.c1)->test.t1.c1, funcs:firstrow(Column#11)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#9, funcs:firstrow(test.t1.c2)->Column#11", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, ?)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestTiFlashExtraColumnPrune", + "Cases": [ + { + "SQL": "explain format = 'brief' select ta.c1 from t1 ta, t1 tb where ta.c1 * ta.c1 > ta.c2 + 10;", + "Plan": [ + "TableReader 80000000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 80000000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 80000000.00 mpp[tiflash] test.t1.c1", + " └─HashJoin 80000000.00 mpp[tiflash] CARTESIAN inner join", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Projection 8000.00 mpp[tiflash] test.t1.c1", + " │ └─Selection 8000.00 mpp[tiflash] gt(mul(test.t1.c1, test.t1.c1), plus(test.t1.c2, 10))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:ta pushed down filter:empty, keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:tb keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from t1 ta, t1 tb where ta.c1 * ta.c1 > ta.c2 + 10;", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#8)->Column#7", + "└─TableReader 1.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#8", + " └─Projection 80000000.00 mpp[tiflash] test.t1.c1", + " └─HashJoin 80000000.00 mpp[tiflash] CARTESIAN inner join", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 8000.00 mpp[tiflash] gt(mul(test.t1.c1, test.t1.c1), plus(test.t1.c2, 10))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:ta pushed down filter:empty, keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:tb keep order:false, stats:pseudo" + ] + } + ] + } +] diff --git a/pkg/planner/core/testdata/runtime_filter_generator_suite_out.json b/pkg/planner/core/testdata/runtime_filter_generator_suite_out.json new file mode 100644 index 0000000000000..158cfc64b6e4b --- /dev/null +++ b/pkg/planner/core/testdata/runtime_filter_generator_suite_out.json @@ -0,0 +1,191 @@ +[ + { + "Name": "TestRuntimeFilterGenerator", + "Cases": [ + { + "SQL": "select /*+ hash_join_build(t1) */ * from t1, t2 where t1.k1=t2.k1 and t2.k2 = 1", + "Plan": [ + "TableReader_32 1.00 root MppVersion: 2, data:ExchangeSender_31", + "└─ExchangeSender_31 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_24 1.00 mpp[tiflash] inner join, equal:[eq(test.t1.k1, test.t2.k1)], runtime filter:0[IN] <- test.t1.k1", + " ├─ExchangeReceiver_28(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender_27 1.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_26 1.00 mpp[tiflash] not(isnull(test.t1.k1))", + " │ └─TableFullScan_25 1.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false", + " └─Selection_30(Probe) 1.00 mpp[tiflash] eq(test.t2.k2, 1), not(isnull(test.t2.k1))", + " └─TableFullScan_29 1.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, runtime filter:0[IN] -> test.t2.k1" + ] + }, + { + "SQL": "select /*+ hash_join_build(t1) */ * from t1, t2 where t1.k1=t2.k1 and t1.k1=t2.k2", + "Plan": [ + "TableReader_32 1.00 root MppVersion: 2, data:ExchangeSender_31", + "└─ExchangeSender_31 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_24 1.00 mpp[tiflash] inner join, equal:[eq(test.t1.k1, test.t2.k1) eq(test.t1.k1, test.t2.k2)], runtime filter:0[IN] <- test.t1.k1, 1[IN] <- test.t1.k1", + " ├─ExchangeReceiver_28(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender_27 1.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_26 1.00 mpp[tiflash] not(isnull(test.t1.k1))", + " │ └─TableFullScan_25 1.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false", + " └─Selection_30(Probe) 1.00 mpp[tiflash] not(isnull(test.t2.k1)), not(isnull(test.t2.k2))", + " └─TableFullScan_29 1.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, runtime filter:0[IN] -> test.t2.k1, 1[IN] -> test.t2.k2" + ] + }, + { + "SQL": "select /*+ shuffle_join(t1, t2) */ * from t1, t2 where t1.k1=t2.k1; -- Global doesn't support", + "Plan": [ + "TableReader_21 1.00 root MppVersion: 2, data:ExchangeSender_20", + "└─ExchangeSender_20 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_19 1.00 mpp[tiflash] inner join, equal:[eq(test.t1.k1, test.t2.k1)]", + " ├─ExchangeReceiver_12(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender_11 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.k1, collate: binary]", + " │ └─Selection_10 1.00 mpp[tiflash] not(isnull(test.t1.k1))", + " │ └─TableFullScan_9 1.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false", + " └─ExchangeReceiver_16(Probe) 1.00 mpp[tiflash] ", + " └─ExchangeSender_15 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t2.k1, collate: binary]", + " └─Selection_14 1.00 mpp[tiflash] not(isnull(test.t2.k1))", + " └─TableFullScan_13 1.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false" + ] + }, + { + "SQL": "select /*+ broadcast_join(t2, t1), hash_join_build(t2) */ * from t2, (select k1 from t1 group by k1) t1 where t1.k1=t2.k1; -- Global doesn't support", + "Plan": [ + "TableReader_32 1.00 root MppVersion: 2, data:ExchangeSender_31", + "└─ExchangeSender_31 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_30 1.00 mpp[tiflash] inner join, equal:[eq(test.t2.k1, test.t1.k1)]", + " ├─ExchangeReceiver_14(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender_13 1.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_12 1.00 mpp[tiflash] not(isnull(test.t2.k1))", + " │ └─TableFullScan_11 1.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false", + " └─Projection_21(Probe) 1.00 mpp[tiflash] test.t1.k1", + " └─HashAgg_15 1.00 mpp[tiflash] group by:test.t1.k1, funcs:firstrow(test.t1.k1)->test.t1.k1", + " └─ExchangeReceiver_20 1.00 mpp[tiflash] ", + " └─ExchangeSender_19 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.k1, collate: binary]", + " └─Selection_18 1.00 mpp[tiflash] not(isnull(test.t1.k1))", + " └─TableFullScan_17 1.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false" + ] + }, + { + "SQL": "select /*+ broadcast_join(t1, t2), hash_join_build(t1) */ * from t1, t2 where t1.k1=t2.k1; -- t1 is build side", + "Plan": [ + "TableReader_19 1.00 root MppVersion: 2, data:ExchangeSender_18", + "└─ExchangeSender_18 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_17 1.00 mpp[tiflash] inner join, equal:[eq(test.t1.k1, test.t2.k1)], runtime filter:0[IN] <- test.t1.k1", + " ├─ExchangeReceiver_12(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender_11 1.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_10 1.00 mpp[tiflash] not(isnull(test.t1.k1))", + " │ └─TableFullScan_9 1.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false", + " └─Selection_14(Probe) 1.00 mpp[tiflash] not(isnull(test.t2.k1))", + " └─TableFullScan_13 1.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false, runtime filter:0[IN] -> test.t2.k1" + ] + }, + { + "SQL": "select * from t1_tikv as t1, t2 where t1.k1=t2.k1; -- Doesn't support hash join in root", + "Plan": [ + "HashJoin_7 1.25 root inner join, equal:[eq(test.t1_tikv.k1, test.t2.k1)]", + "├─TableReader_18(Build) 1.00 root MppVersion: 2, data:ExchangeSender_17", + "│ └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", + "│ └─Selection_16 1.00 mpp[tiflash] not(isnull(test.t2.k1))", + "│ └─TableFullScan_15 1.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false", + "└─TableReader_11(Probe) 9990.00 root data:Selection_10", + " └─Selection_10 9990.00 cop[tikv] not(isnull(test.t1_tikv.k1))", + " └─TableFullScan_9 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ broadcast_join(t1, t2), hash_join_build(t1) */ * from t1, t2 where t1.k1+1=t2.k1; -- Support transform src expression t1.k1+1", + "Plan": [ + "TableReader_22 1.00 root MppVersion: 2, data:ExchangeSender_21", + "└─ExchangeSender_21 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_20 1.00 mpp[tiflash] test.t1.k1, test.t2.k1, test.t2.k2, test.t2.k3", + " └─HashJoin_19 1.00 mpp[tiflash] inner join, equal:[eq(Column#7, test.t2.k1)], runtime filter:0[IN] <- Column#7", + " ├─ExchangeReceiver_14(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender_13 1.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Projection_11 1.00 mpp[tiflash] test.t1.k1, plus(test.t1.k1, 1)->Column#7", + " │ └─TableFullScan_12 1.00 mpp[tiflash] table:t1 keep order:false", + " └─TableFullScan_15(Probe) 1.00 mpp[tiflash] table:t2 keep order:false, runtime filter:0[IN] -> test.t2.k1" + ] + }, + { + "SQL": "select /*+ broadcast_join(t2, t1), hash_join_build(t2) */ * from t2, (select k1, k1+1 as k11 from t1) t1 where t1.k1=t2.k1; -- Only support origin column k1", + "Plan": [ + "TableReader_21 1.00 root MppVersion: 2, data:ExchangeSender_20", + "└─ExchangeSender_20 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_19 1.00 mpp[tiflash] inner join, equal:[eq(test.t2.k1, test.t1.k1)], runtime filter:0[IN] <- test.t2.k1", + " ├─ExchangeReceiver_13(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender_12 1.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_11 1.00 mpp[tiflash] not(isnull(test.t2.k1))", + " │ └─TableFullScan_10 1.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false", + " └─Projection_14(Probe) 1.00 mpp[tiflash] test.t1.k1, plus(test.t1.k1, 1)->Column#7", + " └─Selection_16 1.00 mpp[tiflash] not(isnull(test.t1.k1))", + " └─TableFullScan_15 1.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, runtime filter:0[IN] -> test.t1.k1" + ] + }, + { + "SQL": "select /*+ hash_join_build(t2) */ * from t2, (select k1, k1+1 as k11 from t1) t1 where t1.k11=t2.k1; -- Doesn't support transform column k11", + "Plan": [ + "TableReader_38 0.80 root MppVersion: 2, data:ExchangeSender_37", + "└─ExchangeSender_37 0.80 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_31 0.80 mpp[tiflash] inner join, equal:[eq(test.t2.k1, Column#7)]", + " ├─ExchangeReceiver_35(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender_34 1.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_33 1.00 mpp[tiflash] not(isnull(test.t2.k1))", + " │ └─TableFullScan_32 1.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false", + " └─Projection_36(Probe) 0.80 mpp[tiflash] test.t1.k1, plus(test.t1.k1, 1)->Column#7", + " └─Selection_28 0.80 mpp[tiflash] not(isnull(plus(test.t1.k1, 1)))", + " └─TableFullScan_27 1.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false" + ] + }, + { + "SQL": "select /*+ broadcast_join(t1, t2), hash_join_build(t1) */ * from t1, t2 where t1.k1=t2.k1+1; -- Doesn't support target expression t2.k1+1", + "Plan": [ + "TableReader_21 1.00 root MppVersion: 2, data:ExchangeSender_20", + "└─ExchangeSender_20 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_19 1.00 mpp[tiflash] test.t1.k1, test.t2.k1, test.t2.k2, test.t2.k3", + " └─HashJoin_18 1.00 mpp[tiflash] inner join, equal:[eq(test.t1.k1, Column#7)]", + " ├─ExchangeReceiver_12(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender_11 1.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_10 1.00 mpp[tiflash] table:t1 keep order:false", + " └─Projection_13(Probe) 1.00 mpp[tiflash] test.t2.k1, test.t2.k2, test.t2.k3, plus(test.t2.k1, 1)->Column#7", + " └─TableFullScan_14 1.00 mpp[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "select /*+ broadcast_join(t1, t2), hash_join_build(t1) */ * from t1 right join t2 on t1.k1=t2.k1; -- t2 side couldn't be RF target side, no RF", + "Plan": [ + "TableReader_17 1.00 root MppVersion: 2, data:ExchangeSender_16", + "└─ExchangeSender_16 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_15 1.00 mpp[tiflash] right outer join, equal:[eq(test.t1.k1, test.t2.k1)]", + " ├─ExchangeReceiver_11(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender_10 1.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_9 1.00 mpp[tiflash] not(isnull(test.t1.k1))", + " │ └─TableFullScan_8 1.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false", + " └─TableFullScan_12(Probe) 1.00 mpp[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "select /*+ broadcast_join(t1, t2), hash_join_build(t1) */ * from t1 where t1.k1 not in (select k1 from t2); -- RF could not push to t2 because of anti join", + "Plan": [ + "TableReader_18 0.80 root MppVersion: 2, data:ExchangeSender_17", + "└─ExchangeSender_17 0.80 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_16 0.80 mpp[tiflash] Null-aware anti semi join, equal:[eq(test.t1.k1, test.t2.k1)]", + " ├─ExchangeReceiver_12(Build) 1.00 mpp[tiflash] ", + " │ └─ExchangeSender_11 1.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─TableFullScan_10 1.00 mpp[tiflash] table:t1 keep order:false", + " └─TableFullScan_13(Probe) 1.00 mpp[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "select /*+ hash_join_build(t1) */ * from t1, t2 where t1.k1<=>t2.k1; -- Doesn't support null safe eq predicate", + "Plan": [ + "HashJoin_8 1.00 root CARTESIAN inner join, other cond:nulleq(test.t1.k1, test.t2.k1)", + "├─TableReader_13(Build) 1.00 root MppVersion: 2, data:ExchangeSender_12", + "│ └─ExchangeSender_12 1.00 mpp[tiflash] ExchangeType: PassThrough", + "│ └─TableFullScan_11 1.00 mpp[tiflash] table:t1 keep order:false", + "└─TableReader_18(Probe) 1.00 root MppVersion: 2, data:ExchangeSender_17", + " └─ExchangeSender_17 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_16 1.00 mpp[tiflash] table:t2 keep order:false" + ] + } + ] + } +] diff --git a/pkg/planner/util/fixcontrol/get.go b/pkg/planner/util/fixcontrol/get.go new file mode 100644 index 0000000000000..33d1d083ec37b --- /dev/null +++ b/pkg/planner/util/fixcontrol/get.go @@ -0,0 +1,158 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +// NOTE: For assigning new fix control numbers - use the issue number associated with the fix. +// + +package fixcontrol + +import ( + "strconv" + "strings" +) + +const ( + // Fix33031 controls whether to disallow plan cache for partitioned + // tables (both prepared statments and non-prepared statements) + // See #33031 + Fix33031 uint64 = 33031 + // Fix43817 controls whether to allow optimizer to evaluate non-correlated sub-queries in the optimization phase. + // If it is not allowed, the optimizer will return a particular error when encountering non-correlated sub-queries. + // This fix-control is mainly for Index Advisor. + Fix43817 uint64 = 43817 + // Fix44262 controls whether to allow to use dynamic-mode to access partitioning tables without global-stats (#44262). + Fix44262 uint64 = 44262 + // Fix44389 controls whether to consider non-point ranges of some CNF item when building ranges. + Fix44389 uint64 = 44389 + // Fix44830 controls whether to allow to cache Batch/PointGet from some complex scenarios. + // See #44830 for more details. + Fix44830 uint64 = 44830 + // Fix44823 controls the maximum number of parameters for a query that can be cached in the Plan Cache. + Fix44823 uint64 = 44823 + // Fix44855 controls whether to use a more accurate upper bound when estimating row count of index + // range scan under inner side of index join. + Fix44855 uint64 = 44855 + // Fix45132 controls whether to use access range row count to determine access path on the Skyline pruning. + Fix45132 uint64 = 45132 + // Fix45822 controls whether to eliminate apply operator. + Fix45822 uint64 = 45822 + // Fix45798 controls whether to cache plans that access generated columns. + Fix45798 uint64 = 45798 + // Fix46177 controls whether to explore enforced plans for DataSource if it has already found an unenforced plan. + Fix46177 uint64 = 46177 + // Fix47400 controls whether to allow a rowEst below 1 + Fix47400 uint64 = 47400 + // Fix49736 controls whether to force the optimizer to use plan cache even if there is risky optimization. + // This fix-control is test-only. + Fix49736 uint64 = 49736 + // Fix52869 controls whether to disable the limitation that index merge path won't be generated automatically when + // there exist other single-index access paths that do range scan. + Fix52869 uint64 = 52869 + // Fix54337 controls whether to apply or not range intersection for index access. + Fix54337 uint64 = 54337 +) + +// GetStr fetches the given key from the fix control map as a string type. +func GetStr(fixControlMap map[uint64]string, key uint64) (value string, exists bool) { + if fixControlMap == nil { + return "", false + } + rawValue, ok := fixControlMap[key] + if !ok { + return "", false + } + return rawValue, true +} + +// GetStrWithDefault fetches the given key from the fix control map as a string type, +// and a default value would be returned when fail to fetch the expected key. +func GetStrWithDefault(fixControlMap map[uint64]string, key uint64, defaultVal string) string { + value, exists := GetStr(fixControlMap, key) + if !exists { + return defaultVal + } + return value +} + +// GetBool fetches the given key from the fix control map as a boolean type. +func GetBool(fixControlMap map[uint64]string, key uint64) (value bool, exists bool) { + if fixControlMap == nil { + return false, false + } + rawValue, ok := fixControlMap[key] + if !ok { + return false, false + } + // The same as TiDBOptOn in sessionctx/variable. + value = strings.EqualFold(rawValue, "ON") || rawValue == "1" + return value, true +} + +// GetBoolWithDefault fetches the given key from the fix control map as a boolean type, +// and a default value would be returned when fail to fetch the expected key. +func GetBoolWithDefault(fixControlMap map[uint64]string, key uint64, defaultVal bool) bool { + value, exists := GetBool(fixControlMap, key) + if !exists { + return defaultVal + } + return value +} + +// GetInt fetches the given key from the fix control map as an uint64 type. +func GetInt(fixControlMap map[uint64]string, key uint64) (value int64, exists bool, parseErr error) { + if fixControlMap == nil { + return 0, false, nil + } + rawValue, ok := fixControlMap[key] + if !ok { + return 0, false, nil + } + // The same as TidbOptInt64 in sessionctx/variable. + value, parseErr = strconv.ParseInt(rawValue, 10, 64) + return value, true, parseErr +} + +// GetIntWithDefault fetches the given key from the fix control map as an uint64 type, +// // and a default value would be returned when fail to fetch the expected key. +func GetIntWithDefault(fixControlMap map[uint64]string, key uint64, defaultVal int64) int64 { + value, exists, err := GetInt(fixControlMap, key) + if !exists || err != nil { + return defaultVal + } + return value +} + +// GetFloat fetches the given key from the fix control map as a float64 type. +func GetFloat(fixControlMap map[uint64]string, key uint64) (value float64, exists bool, parseErr error) { + if fixControlMap == nil { + return 0, false, nil + } + rawValue, ok := fixControlMap[key] + if !ok { + return 0, false, nil + } + // The same as tidbOptFloat64 in sessionctx/variable. + value, parseErr = strconv.ParseFloat(rawValue, 64) + return value, true, parseErr +} + +// GetFloatWithDefault fetches the given key from the fix control map as a float64 type, +// // and a default value would be returned when fail to fetch the expected key. +func GetFloatWithDefault(fixControlMap map[uint64]string, key uint64, defaultVal float64) float64 { + value, exists, err := GetFloat(fixControlMap, key) + if !exists || err != nil { + return defaultVal + } + return value +} diff --git a/pkg/statistics/handle/globalstats/global_stats_test.go b/pkg/statistics/handle/globalstats/global_stats_test.go new file mode 100644 index 0000000000000..828ff6e35b8d3 --- /dev/null +++ b/pkg/statistics/handle/globalstats/global_stats_test.go @@ -0,0 +1,969 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package globalstats_test + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/session" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/stretchr/testify/require" +) + +func TestShowGlobalStatsWithAsyncMergeGlobal(t *testing.T) { + testShowGlobalStats(t, true) +} + +func TestShowGlobalStatsWithoutAsyncMergeGlobal(t *testing.T) { + testShowGlobalStats(t, false) +} + +func testShowGlobalStats(t *testing.T, isAsync bool) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@session.tidb_analyze_version = 0") + if isAsync { + tk.MustExec("set @@global.tidb_enable_async_merge_global_stats = 0") + } else { + tk.MustExec("set @@global.tidb_enable_async_merge_global_stats = 1") + } + tk.MustExec("drop table if exists t") + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") + tk.MustExec("create table t (a int, key(a)) partition by hash(a) partitions 2") + tk.MustExec("insert into t values (1), (2), (3), (4)") + tk.MustExec("analyze table t with 1 buckets") + require.Len(t, tk.MustQuery("show stats_meta").Rows(), 2) + require.Len(t, tk.MustQuery("show stats_meta where partition_name='global'").Rows(), 0) + require.Len(t, tk.MustQuery("show stats_buckets").Rows(), 4) // 2 partitions * (1 for the column_a and 1 for the index_a) + require.Len(t, tk.MustQuery("show stats_buckets where partition_name='global'").Rows(), 0) + require.Len(t, tk.MustQuery("show stats_histograms").Rows(), 4) + require.Len(t, tk.MustQuery("show stats_histograms where partition_name='global'").Rows(), 0) + require.Len(t, tk.MustQuery("show stats_healthy").Rows(), 2) + require.Len(t, tk.MustQuery("show stats_healthy where partition_name='global'").Rows(), 0) + + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("analyze table t with 0 topn, 1 buckets") + require.Len(t, tk.MustQuery("show stats_meta").Rows(), 3) + require.Len(t, tk.MustQuery("show stats_meta where partition_name='global'").Rows(), 1) + require.Len(t, tk.MustQuery("show stats_buckets").Rows(), 6) + require.Len(t, tk.MustQuery("show stats_buckets where partition_name='global'").Rows(), 2) + require.Len(t, tk.MustQuery("show stats_histograms").Rows(), 6) + require.Len(t, tk.MustQuery("show stats_histograms where partition_name='global'").Rows(), 2) + require.Len(t, tk.MustQuery("show stats_healthy").Rows(), 3) + require.Len(t, tk.MustQuery("show stats_healthy where partition_name='global'").Rows(), 1) +} + +func simpleTest(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)) partition by hash(a) partitions 10") + tk.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (8), (10), (20), (30)") + tk.MustExec("analyze table t with 0 topn, 1 buckets") +} + +func TestGlobalStatsPanicInIOWorker(t *testing.T) { + fpName := "github.com/pingcap/tidb/pkg/statistics/handle/globalstats/PanicInIOWorker" + require.NoError(t, failpoint.Enable(fpName, "panic(\"inject panic\")")) + defer func() { + require.NoError(t, failpoint.Disable(fpName)) + }() + simpleTest(t) +} + +func TestGlobalStatsWithCMSketchErr(t *testing.T) { + fpName := "github.com/pingcap/tidb/pkg/statistics/handle/globalstats/dealCMSketchErr" + require.NoError(t, failpoint.Enable(fpName, `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable(fpName)) + }() + simpleTest(t) +} + +func TestGlobalStatsWithHistogramAndTopNErr(t *testing.T) { + fpName := "github.com/pingcap/tidb/pkg/statistics/handle/globalstats/dealHistogramAndTopNErr" + require.NoError(t, failpoint.Enable(fpName, `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable(fpName)) + }() + simpleTest(t) +} + +func TestGlobalStatsPanicInCPUWorker(t *testing.T) { + fpName := "github.com/pingcap/tidb/pkg/statistics/handle/globalstats/PanicInCPUWorker" + require.NoError(t, failpoint.Enable(fpName, "panic(\"inject panic\")")) + defer func() { + require.NoError(t, failpoint.Disable(fpName)) + }() + simpleTest(t) +} + +func TestGlobalStatsPanicSametime(t *testing.T) { + fpName := "github.com/pingcap/tidb/pkg/statistics/handle/globalstats/PanicSameTime" + require.NoError(t, failpoint.Enable(fpName, `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable(fpName)) + }() + simpleTest(t) +} + +func TestGlobalStatsErrorSametime(t *testing.T) { + fpName := "github.com/pingcap/tidb/pkg/statistics/handle/globalstats/ErrorSameTime" + require.NoError(t, failpoint.Enable(fpName, `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable(fpName)) + }() + simpleTest(t) +} + +func TestBuildGlobalLevelStats(t *testing.T) { + store := testkit.CreateMockStore(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t, t1;") + testKit.MustExec("set @@tidb_analyze_version = 2") + testKit.MustExec("set @@tidb_partition_prune_mode = 'static';") + testKit.MustExec("create table t(a int, b int, c int) PARTITION BY HASH(a) PARTITIONS 3;") + testKit.MustExec("create table t1(a int);") + testKit.MustExec("insert into t values(1,1,1),(3,12,3),(4,20,4),(2,7,2),(5,21,5);") + testKit.MustExec("insert into t1 values(1),(3),(4),(2),(5);") + testKit.MustExec("create index idx_t_ab on t(a, b);") + testKit.MustExec("create index idx_t_b on t(b);") + testKit.MustExec("select * from t where c = 0") + testKit.MustExec("select * from t1 where a = 0") + do, err := session.GetDomain(store) + require.NoError(t, err) + statsHandle := do.StatsHandle() + require.NoError(t, statsHandle.DumpColStatsUsageToKV()) + testKit.MustExec("analyze table t, t1;") + result := testKit.MustQuery("show stats_meta where table_name = 't';").Sort() + require.Len(t, result.Rows(), 3) + require.Equal(t, "1", result.Rows()[0][5]) + require.Equal(t, "2", result.Rows()[1][5]) + require.Equal(t, "2", result.Rows()[2][5]) + result = testKit.MustQuery("show stats_histograms where table_name = 't';").Sort() + require.Len(t, result.Rows(), 15) + + result = testKit.MustQuery("show stats_meta where table_name = 't1';").Sort() + require.Len(t, result.Rows(), 1) + require.Equal(t, "5", result.Rows()[0][5]) + result = testKit.MustQuery("show stats_histograms where table_name = 't1';").Sort() + require.Len(t, result.Rows(), 1) + + // Test the 'dynamic' mode + testKit.MustExec("set @@tidb_partition_prune_mode = 'dynamic';") + testKit.MustExec("analyze table t, t1;") + result = testKit.MustQuery("show stats_meta where table_name = 't'").Sort() + require.Len(t, result.Rows(), 4) + require.Equal(t, "5", result.Rows()[0][5]) + require.Equal(t, "1", result.Rows()[1][5]) + require.Equal(t, "2", result.Rows()[2][5]) + require.Equal(t, "2", result.Rows()[3][5]) + result = testKit.MustQuery("show stats_histograms where table_name = 't';").Sort() + require.Len(t, result.Rows(), 20) + + result = testKit.MustQuery("show stats_meta where table_name = 't1';").Sort() + require.Len(t, result.Rows(), 1) + require.Equal(t, "5", result.Rows()[0][5]) + result = testKit.MustQuery("show stats_histograms where table_name = 't1';").Sort() + require.Len(t, result.Rows(), 1) + + testKit.MustExec("analyze table t index idx_t_ab, idx_t_b;") + result = testKit.MustQuery("show stats_meta where table_name = 't'").Sort() + require.Len(t, result.Rows(), 4) + require.Equal(t, "5", result.Rows()[0][5]) + require.Equal(t, "1", result.Rows()[1][5]) + require.Equal(t, "2", result.Rows()[2][5]) + require.Equal(t, "2", result.Rows()[3][5]) + result = testKit.MustQuery("show stats_histograms where table_name = 't';").Sort() + require.Len(t, result.Rows(), 20) +} + +func TestGlobalStatsHealthy(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec(` +create table t ( + a int, + key(a) +) +partition by range (a) ( + partition p0 values less than (10), + partition p1 values less than (20) +)`) + + checkModifyAndCount := func(gModify, gCount, p0Modify, p0Count, p1Modify, p1Count int) { + rs := tk.MustQuery("show stats_meta").Rows() + require.Equal(t, fmt.Sprintf("%v", gModify), rs[0][4].(string)) // global.modify_count + require.Equal(t, fmt.Sprintf("%v", gCount), rs[0][5].(string)) // global.row_count + require.Equal(t, fmt.Sprintf("%v", p0Modify), rs[1][4].(string)) // p0.modify_count + require.Equal(t, fmt.Sprintf("%v", p0Count), rs[1][5].(string)) // p0.row_count + require.Equal(t, fmt.Sprintf("%v", p1Modify), rs[2][4].(string)) // p1.modify_count + require.Equal(t, fmt.Sprintf("%v", p1Count), rs[2][5].(string)) // p1.row_count + } + checkHealthy := func(gH, p0H, p1H int) { + tk.MustQuery("show stats_healthy").Check(testkit.Rows( + fmt.Sprintf("test t global %v", gH), + fmt.Sprintf("test t p0 %v", p0H), + fmt.Sprintf("test t p1 %v", p1H))) + } + + tk.MustExec("set @@tidb_analyze_version=2") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + tk.MustExec("analyze table t") + checkModifyAndCount(0, 0, 0, 0, 0, 0) + checkHealthy(100, 100, 100) + + tk.MustExec("insert into t values (1), (2)") // update p0 + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true)) + require.NoError(t, dom.StatsHandle().Update(context.Background(), dom.InfoSchema())) + checkModifyAndCount(2, 2, 2, 2, 0, 0) + checkHealthy(0, 0, 100) + + tk.MustExec("insert into t values (11), (12), (13), (14)") // update p1 + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true)) + require.NoError(t, dom.StatsHandle().Update(context.Background(), dom.InfoSchema())) + checkModifyAndCount(6, 6, 2, 2, 4, 4) + checkHealthy(0, 0, 0) + + tk.MustExec("analyze table t") + checkModifyAndCount(0, 6, 0, 2, 0, 4) + checkHealthy(100, 100, 100) + + tk.MustExec("insert into t values (4), (5), (15), (16)") // update p0 and p1 together + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true)) + require.NoError(t, dom.StatsHandle().Update(context.Background(), dom.InfoSchema())) + checkModifyAndCount(4, 10, 2, 4, 2, 6) + checkHealthy(33, 0, 50) +} + +func TestGlobalStatsData(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec(` +create table t ( + a int, + key(a) +) +partition by range (a) ( + partition p0 values less than (10), + partition p1 values less than (20) +)`) + tk.MustExec("set @@tidb_analyze_version=2") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + tk.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (6), (null), (11), (12), (13), (14), (15), (16), (17), (18), (19), (19)") + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true)) + tk.MustExec("analyze table t with 0 topn, 2 buckets") + + tk.MustQuery("select modify_count, count from mysql.stats_meta order by table_id asc").Check( + testkit.Rows("0 18", "0 8", "0 10")) // global row-count = sum(partition row-count) + + // distinct, null_count, tot_col_size should be the sum of their values in partition-stats, and correlation should be 0 + tk.MustQuery("select distinct_count, null_count, tot_col_size, correlation=0 from mysql.stats_histograms where is_index=0 order by table_id asc").Check( + testkit.Rows("15 1 17 1", "6 1 7 0", "9 0 10 0")) + tk.MustQuery("select distinct_count, null_count, tot_col_size, correlation=0 from mysql.stats_histograms where is_index=1 order by table_id asc").Check( + testkit.Rows("15 1 0 1", "6 1 7 1", "9 0 10 1")) + + tk.MustQuery("show stats_buckets where is_index=0").Check( + // db table partition col is_idx bucket_id count repeats lower upper ndv + testkit.Rows("test t global a 0 0 7 2 1 6 0", + "test t global a 0 1 17 2 6 19 0", + "test t p0 a 0 0 4 1 1 4 0", + "test t p0 a 0 1 7 2 5 6 0", + "test t p1 a 0 0 6 1 11 16 0", + "test t p1 a 0 1 10 2 17 19 0")) + tk.MustQuery("show stats_buckets where is_index=1").Check( + testkit.Rows("test t global a 1 0 7 2 1 6 0", + "test t global a 1 1 17 2 6 19 0", + "test t p0 a 1 0 4 1 1 4 0", + "test t p0 a 1 1 7 2 5 6 0", + "test t p1 a 1 0 6 1 11 16 0", + "test t p1 a 1 1 10 2 17 19 0")) +} + +func TestGlobalStatsData2(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + testGlobalStats2(t, tk, dom) +} + +func TestGlobalStatsData2WithConcurrency(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_merge_partition_stats_concurrency=2") + defer func() { + tk.MustExec("set global tidb_merge_partition_stats_concurrency=1") + }() + testGlobalStats2(t, tk, dom) +} + +func TestGlobalStatsData3(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + tk.MustExec("set @@tidb_analyze_version=2") + + // index(int, int) + tk.MustExec("drop table if exists tintint") + tk.MustExec("create table tintint (a int, b int, key(a, b)) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20))") + tk.MustExec(`insert into tintint values ` + + `(1, 1), (1, 2), (2, 1), (2, 2), (2, 3), (2, 3), (3, 1), (3, 1), (3, 1),` + // values in p0 + `(11, 1), (12, 1), (12, 2), (13, 1), (13, 1), (13, 2), (13, 2), (13, 2)`) // values in p1 + tk.MustExec("analyze table tintint with 2 topn, 2 buckets") + + rs := tk.MustQuery("show stats_meta where table_name='tintint'").Rows() + require.Equal(t, "17", rs[0][5].(string)) // g.total = p0.total + p1.total + require.Equal(t, "9", rs[1][5].(string)) + require.Equal(t, "8", rs[2][5].(string)) + + tk.MustQuery("show stats_topn where table_name='tintint' and is_index=1").Check(testkit.Rows( + "test tintint global a 1 (3, 1) 3", + "test tintint global a 1 (13, 2) 3", + "test tintint p0 a 1 (2, 3) 2", + "test tintint p0 a 1 (3, 1) 3", + "test tintint p1 a 1 (13, 1) 2", + "test tintint p1 a 1 (13, 2) 3")) + + tk.MustQuery("show stats_buckets where table_name='tintint' and is_index=1").Check(testkit.Rows( + "test tintint global a 1 0 6 2 (1, 1) (2, 3) 0", // (2, 3) is popped into it + "test tintint global a 1 1 11 2 (13, 1) (13, 1) 0", // (13, 1) is popped into it + "test tintint p0 a 1 0 3 1 (1, 1) (2, 1) 0", + "test tintint p0 a 1 1 4 1 (2, 2) (2, 2) 0", + "test tintint p1 a 1 0 2 1 (11, 1) (12, 1) 0", + "test tintint p1 a 1 1 3 1 (12, 2) (12, 2) 0")) + + rs = tk.MustQuery("show stats_histograms where table_name='tintint' and is_index=1").Rows() + require.Equal(t, "11", rs[0][6].(string)) // g.ndv = p0.ndv + p1.ndv + require.Equal(t, "6", rs[1][6].(string)) + require.Equal(t, "5", rs[2][6].(string)) + + // index(int, string) + tk.MustExec("drop table if exists tintstr") + tk.MustExec("create table tintstr (a int, b varchar(32), key(a, b)) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20))") + tk.MustExec(`insert into tintstr values ` + + `(1, '1'), (1, '2'), (2, '1'), (2, '2'), (2, '3'), (2, '3'), (3, '1'), (3, '1'), (3, '1'),` + // values in p0 + `(11, '1'), (12, '1'), (12, '2'), (13, '1'), (13, '1'), (13, '2'), (13, '2'), (13, '2')`) // values in p1 + tk.MustExec("analyze table tintstr with 2 topn, 2 buckets") + + rs = tk.MustQuery("show stats_meta where table_name='tintstr'").Rows() + require.Equal(t, "17", rs[0][5].(string)) // g.total = p0.total + p1.total + require.Equal(t, "9", rs[1][5].(string)) + require.Equal(t, "8", rs[2][5].(string)) + + tk.MustQuery("show stats_topn where table_name='tintstr' and is_index=1").Check(testkit.Rows( + "test tintstr global a 1 (3, 1) 3", + "test tintstr global a 1 (13, 2) 3", + "test tintstr p0 a 1 (2, 3) 2", + "test tintstr p0 a 1 (3, 1) 3", + "test tintstr p1 a 1 (13, 1) 2", + "test tintstr p1 a 1 (13, 2) 3")) + + tk.MustQuery("show stats_buckets where table_name='tintstr' and is_index=1").Check(testkit.Rows( + "test tintstr global a 1 0 6 2 (1, 1) (2, 3) 0", // (2, 3) is popped into it + "test tintstr global a 1 1 11 2 (13, 1) (13, 1) 0", // (13, 1) is popped into it + "test tintstr p0 a 1 0 3 1 (1, 1) (2, 1) 0", + "test tintstr p0 a 1 1 4 1 (2, 2) (2, 2) 0", + "test tintstr p1 a 1 0 2 1 (11, 1) (12, 1) 0", + "test tintstr p1 a 1 1 3 1 (12, 2) (12, 2) 0")) + + rs = tk.MustQuery("show stats_histograms where table_name='tintstr' and is_index=1").Rows() + require.Equal(t, "11", rs[0][6].(string)) // g.ndv = p0.ndv + p1.ndv + require.Equal(t, "6", rs[1][6].(string)) + require.Equal(t, "5", rs[2][6].(string)) + + // index(int, double) + tk.MustExec("drop table if exists tintdouble") + tk.MustExec("create table tintdouble (a int, b double, key(a, b)) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20))") + tk.MustExec(`insert into tintdouble values ` + + `(1, 1), (1, 2), (2, 1), (2, 2), (2, 3), (2, 3), (3, 1), (3, 1), (3, 1),` + // values in p0 + `(11, 1), (12, 1), (12, 2), (13, 1), (13, 1), (13, 2), (13, 2), (13, 2)`) // values in p1 + tk.MustExec("analyze table tintdouble with 2 topn, 2 buckets") + + rs = tk.MustQuery("show stats_meta where table_name='tintdouble'").Rows() + require.Equal(t, "17", rs[0][5].(string)) // g.total = p0.total + p1.total + require.Equal(t, "9", rs[1][5].(string)) + require.Equal(t, "8", rs[2][5].(string)) + + tk.MustQuery("show stats_topn where table_name='tintdouble' and is_index=1").Check(testkit.Rows( + "test tintdouble global a 1 (3, 1) 3", + "test tintdouble global a 1 (13, 2) 3", + "test tintdouble p0 a 1 (2, 3) 2", + "test tintdouble p0 a 1 (3, 1) 3", + "test tintdouble p1 a 1 (13, 1) 2", + "test tintdouble p1 a 1 (13, 2) 3")) + + tk.MustQuery("show stats_buckets where table_name='tintdouble' and is_index=1").Check(testkit.Rows( + "test tintdouble global a 1 0 6 2 (1, 1) (2, 3) 0", // (2, 3) is popped into it + "test tintdouble global a 1 1 11 2 (13, 1) (13, 1) 0", // (13, 1) is popped into it + "test tintdouble p0 a 1 0 3 1 (1, 1) (2, 1) 0", + "test tintdouble p0 a 1 1 4 1 (2, 2) (2, 2) 0", + "test tintdouble p1 a 1 0 2 1 (11, 1) (12, 1) 0", + "test tintdouble p1 a 1 1 3 1 (12, 2) (12, 2) 0")) + + rs = tk.MustQuery("show stats_histograms where table_name='tintdouble' and is_index=1").Rows() + require.Equal(t, "11", rs[0][6].(string)) // g.ndv = p0.ndv + p1.ndv + require.Equal(t, "6", rs[1][6].(string)) + require.Equal(t, "5", rs[2][6].(string)) + + // index(double, decimal) + tk.MustExec("drop table if exists tdoubledecimal") + tk.MustExec("create table tdoubledecimal (a int, b decimal(30, 2), key(a, b)) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20))") + tk.MustExec(`insert into tdoubledecimal values ` + + `(1, 1), (1, 2), (2, 1), (2, 2), (2, 3), (2, 3), (3, 1), (3, 1), (3, 1),` + // values in p0 + `(11, 1), (12, 1), (12, 2), (13, 1), (13, 1), (13, 2), (13, 2), (13, 2)`) // values in p1 + tk.MustExec("analyze table tdoubledecimal with 2 topn, 2 buckets") + + rs = tk.MustQuery("show stats_meta where table_name='tdoubledecimal'").Rows() + require.Equal(t, "17", rs[0][5].(string)) // g.total = p0.total + p1.total + require.Equal(t, "9", rs[1][5].(string)) + require.Equal(t, "8", rs[2][5].(string)) + + tk.MustQuery("show stats_topn where table_name='tdoubledecimal' and is_index=1").Check(testkit.Rows( + "test tdoubledecimal global a 1 (3, 1.00) 3", + "test tdoubledecimal global a 1 (13, 2.00) 3", + "test tdoubledecimal p0 a 1 (2, 3.00) 2", + "test tdoubledecimal p0 a 1 (3, 1.00) 3", + "test tdoubledecimal p1 a 1 (13, 1.00) 2", + "test tdoubledecimal p1 a 1 (13, 2.00) 3")) + + tk.MustQuery("show stats_buckets where table_name='tdoubledecimal' and is_index=1").Check(testkit.Rows( + "test tdoubledecimal global a 1 0 6 2 (1, 1.00) (2, 3.00) 0", // (2, 3) is popped into it + "test tdoubledecimal global a 1 1 11 2 (13, 1.00) (13, 1.00) 0", // (13, 1) is popped into it + "test tdoubledecimal p0 a 1 0 3 1 (1, 1.00) (2, 1.00) 0", + "test tdoubledecimal p0 a 1 1 4 1 (2, 2.00) (2, 2.00) 0", + "test tdoubledecimal p1 a 1 0 2 1 (11, 1.00) (12, 1.00) 0", + "test tdoubledecimal p1 a 1 1 3 1 (12, 2.00) (12, 2.00) 0")) + + rs = tk.MustQuery("show stats_histograms where table_name='tdoubledecimal' and is_index=1").Rows() + require.Equal(t, "11", rs[0][6].(string)) // g.ndv = p0.ndv + p1.ndv + require.Equal(t, "6", rs[1][6].(string)) + require.Equal(t, "5", rs[2][6].(string)) + + // index(string, datetime) + tk.MustExec("drop table if exists tstrdt") + tk.MustExec("create table tstrdt (a int, b datetime, key(a, b)) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20))") + tk.MustExec(`insert into tstrdt values ` + + `(1, '2000-01-01'), (1, '2000-01-02'), (2, '2000-01-01'), (2, '2000-01-02'), (2, '2000-01-03'), (2, '2000-01-03'), (3, '2000-01-01'), (3, '2000-01-01'), (3, '2000-01-01'),` + // values in p0 + `(11, '2000-01-01'), (12, '2000-01-01'), (12, '2000-01-02'), (13, '2000-01-01'), (13, '2000-01-01'), (13, '2000-01-02'), (13, '2000-01-02'), (13, '2000-01-02')`) // values in p1 + tk.MustExec("analyze table tstrdt with 2 topn, 2 buckets") + + rs = tk.MustQuery("show stats_meta where table_name='tstrdt'").Rows() + require.Equal(t, "17", rs[0][5].(string)) // g.total = p0.total + p1.total + require.Equal(t, "9", rs[1][5].(string)) + require.Equal(t, "8", rs[2][5].(string)) + + tk.MustQuery("show stats_topn where table_name='tstrdt' and is_index=1").Check(testkit.Rows( + "test tstrdt global a 1 (3, 2000-01-01 00:00:00) 3", + "test tstrdt global a 1 (13, 2000-01-02 00:00:00) 3", + "test tstrdt p0 a 1 (2, 2000-01-03 00:00:00) 2", + "test tstrdt p0 a 1 (3, 2000-01-01 00:00:00) 3", + "test tstrdt p1 a 1 (13, 2000-01-01 00:00:00) 2", + "test tstrdt p1 a 1 (13, 2000-01-02 00:00:00) 3")) + + tk.MustQuery("show stats_buckets where table_name='tstrdt' and is_index=1").Check(testkit.Rows( + "test tstrdt global a 1 0 6 2 (1, 2000-01-01 00:00:00) (2, 2000-01-03 00:00:00) 0", // (2, 3) is popped into it + "test tstrdt global a 1 1 11 2 (13, 2000-01-01 00:00:00) (13, 2000-01-01 00:00:00) 0", // (13, 1) is popped into it + "test tstrdt p0 a 1 0 3 1 (1, 2000-01-01 00:00:00) (2, 2000-01-01 00:00:00) 0", + "test tstrdt p0 a 1 1 4 1 (2, 2000-01-02 00:00:00) (2, 2000-01-02 00:00:00) 0", + "test tstrdt p1 a 1 0 2 1 (11, 2000-01-01 00:00:00) (12, 2000-01-01 00:00:00) 0", + "test tstrdt p1 a 1 1 3 1 (12, 2000-01-02 00:00:00) (12, 2000-01-02 00:00:00) 0")) + + rs = tk.MustQuery("show stats_histograms where table_name='tstrdt' and is_index=1").Rows() + require.Equal(t, "11", rs[0][6].(string)) // g.ndv = p0.ndv + p1.ndv + require.Equal(t, "6", rs[1][6].(string)) + require.Equal(t, "5", rs[2][6].(string)) +} + +func TestGlobalStatsVersion(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec(` +create table t ( + a int +) +partition by range (a) ( + partition p0 values less than (10), + partition p1 values less than (20) +)`) + require.NoError(t, dom.StatsHandle().HandleDDLEvent(<-dom.StatsHandle().DDLEventCh())) + tk.MustExec("insert into t values (1), (5), (null), (11), (15)") + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true)) + + tk.MustExec("set @@tidb_partition_prune_mode='static'") + tk.MustExec("set @@session.tidb_analyze_version=1") + tk.MustExec("analyze table t") // both p0 and p1 are in ver1 + require.Len(t, tk.MustQuery("show stats_meta").Rows(), 2) + + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + tk.MustExec("set @@session.tidb_analyze_version=1") + err := tk.ExecToErr("analyze table t") // try to build global-stats on ver1 + require.NoError(t, err) + + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + tk.MustExec("set @@session.tidb_analyze_version=2") + err = tk.ExecToErr("analyze table t partition p1") // only analyze p1 to let it in ver2 while p0 is in ver1 + require.NoError(t, err) + + tk.MustExec("analyze table t") // both p0 and p1 are in ver2 + require.Len(t, tk.MustQuery("show stats_meta").Rows(), 3) + + // If we already have global-stats, we can get the latest global-stats by analyzing the newly added partition. + tk.MustExec("alter table t add partition (partition p2 values less than (30))") + tk.MustExec("insert t values (13), (14), (22), (23)") + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true)) + tk.MustExec("analyze table t partition p2") // it will success since p0 and p1 are both in ver2 + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true)) + do := dom + is := do.InfoSchema() + h := do.StatsHandle() + require.NoError(t, h.Update(context.Background(), is)) + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tableInfo := tbl.Meta() + globalStats := h.GetTableStats(tableInfo) + // global.count = p0.count(3) + p1.count(4) + p2.count(2) + // modify count is 2 because we didn't analyze p1 after the second insert + require.Equal(t, int64(9), globalStats.RealtimeCount) + require.Equal(t, int64(2), globalStats.ModifyCount) + + tk.MustExec("analyze table t partition p1;") + globalStats = h.GetTableStats(tableInfo) + // global.count = p0.count(3) + p1.count(4) + p2.count(4) + // The value of modify count is 0 now. + require.Equal(t, int64(9), globalStats.RealtimeCount) + require.Equal(t, int64(0), globalStats.ModifyCount) + + tk.MustExec("alter table t drop partition p2;") + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true)) + tk.MustExec("analyze table t;") + globalStats = h.GetTableStats(tableInfo) + // global.count = p0.count(3) + p1.count(4) + require.Equal(t, int64(7), globalStats.RealtimeCount) +} + +func TestDDLPartition4GlobalStats(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("set @@session.tidb_analyze_version=2") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + tk.MustExec(`create table t (a int) partition by range (a) ( + partition p0 values less than (10), + partition p1 values less than (20), + partition p2 values less than (30), + partition p3 values less than (40), + partition p4 values less than (50), + partition p5 values less than (60) + )`) + do := dom + is := do.InfoSchema() + h := do.StatsHandle() + err := h.HandleDDLEvent(<-h.DDLEventCh()) + require.NoError(t, err) + require.NoError(t, h.Update(context.Background(), is)) + tk.MustExec("insert into t values (1), (2), (3), (4), (5), " + + "(11), (21), (31), (41), (51)," + + "(12), (22), (32), (42), (52);") + require.NoError(t, h.DumpStatsDeltaToKV(true)) + require.NoError(t, h.Update(context.Background(), is)) + tk.MustExec("analyze table t") + result := tk.MustQuery("show stats_meta where table_name = 't';").Rows() + require.Len(t, result, 7) + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tableInfo := tbl.Meta() + globalStats := h.GetTableStats(tableInfo) + require.Equal(t, int64(15), globalStats.RealtimeCount) + + tk.MustExec("alter table t truncate partition p2, p4;") + require.NoError(t, h.DumpStatsDeltaToKV(true)) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + require.NoError(t, h.Update(context.Background(), is)) + // We will update the global-stats after the truncate operation. + globalStats = h.GetTableStats(tableInfo) + require.Equal(t, int64(11), globalStats.RealtimeCount) + + tk.MustExec("analyze table t;") + result = tk.MustQuery("show stats_meta where table_name = 't';").Rows() + // The truncate operation only delete the data from the partition p2 and p4. It will not delete the partition-stats. + require.Len(t, result, 7) + // The result for the globalStats.count will be right now + globalStats = h.GetTableStats(tableInfo) + require.Equal(t, int64(11), globalStats.RealtimeCount) +} + +func TestGlobalStatsNDV(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec(`CREATE TABLE t ( a int, key(a) ) + PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (10), + PARTITION p1 VALUES LESS THAN (20), + PARTITION p2 VALUES LESS THAN (30), + PARTITION p3 VALUES LESS THAN (40))`) + + checkNDV := func(ndvs ...int) { // g, p0, ..., p3 + tk.MustExec("analyze table t") + rs := tk.MustQuery(`show stats_histograms where is_index=1`).Rows() + require.Len(t, rs, 5) + for i, ndv := range ndvs { + require.Equal(t, fmt.Sprintf("%v", ndv), rs[i][6].(string)) + } + } + + // all partitions are empty + checkNDV(0, 0, 0, 0, 0) + + // p0 has data while others are empty + tk.MustExec("insert into t values (1), (2), (3)") + checkNDV(3, 3, 0, 0, 0) + + // p0, p1, p2 have data while p3 is empty + tk.MustExec("insert into t values (11), (12), (13), (21), (22), (23)") + checkNDV(9, 3, 3, 3, 0) + + // all partitions are not empty + tk.MustExec("insert into t values (31), (32), (33), (34)") + checkNDV(13, 3, 3, 3, 4) + + // insert some duplicated records + tk.MustExec("insert into t values (31), (33), (34)") + tk.MustExec("insert into t values (1), (2), (3)") + checkNDV(13, 3, 3, 3, 4) +} + +func TestGlobalStatsIndexNDV(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + checkNDV := func(tbl string, g int, ps ...int) { // g, p0, ..., p3 + tk.MustExec("analyze table " + tbl) + rs := tk.MustQuery(fmt.Sprintf(`show stats_histograms where is_index=1 and table_name='%v'`, tbl)).Rows() + require.Len(t, rs, 1+len(ps)) // 1(global) + number of partitions + require.Equal(t, fmt.Sprintf("%v", g), rs[0][6].(string)) // global + for i, ndv := range ps { + require.Equal(t, fmt.Sprintf("%v", ndv), rs[i+1][6].(string)) + } + } + + // int + tk.MustExec("drop table if exists tint") + tk.MustExec(`CREATE TABLE tint ( a int, b int, key(b) ) + PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (10), + PARTITION p1 VALUES LESS THAN (20))`) + tk.MustExec("insert into tint values (1, 1), (1, 2), (1, 3)") // p0.b: [1, 2, 3], p1.b: [] + checkNDV("tint", 3, 3, 0) + tk.MustExec("insert into tint values (11, 1), (11, 2), (11, 3)") // p0.b: [1, 2, 3], p1.b: [1, 2, 3] + checkNDV("tint", 3, 3, 3) + tk.MustExec("insert into tint values (11, 4), (11, 5), (11, 6)") // p0.b: [1, 2, 3], p1.b: [1, 2, 3, 4, 5, 6] + checkNDV("tint", 6, 3, 6) + tk.MustExec("insert into tint values (1, 4), (1, 5), (1, 6), (1, 7), (1, 8)") // p0.b: [1, 2, 3, 4, 5, 6, 7, 8], p1.b: [1, 2, 3, 4, 5, 6] + checkNDV("tint", 8, 8, 6) + + // double + tk.MustExec("drop table if exists tdouble") + tk.MustExec(`CREATE TABLE tdouble ( a int, b double, key(b) ) + PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (10), + PARTITION p1 VALUES LESS THAN (20))`) + tk.MustExec("insert into tdouble values (1, 1.1), (1, 2.2), (1, 3.3)") // p0.b: [1, 2, 3], p1.b: [] + checkNDV("tdouble", 3, 3, 0) + tk.MustExec("insert into tdouble values (11, 1.1), (11, 2.2), (11, 3.3)") // p0.b: [1, 2, 3], p1.b: [1, 2, 3] + checkNDV("tdouble", 3, 3, 3) + tk.MustExec("insert into tdouble values (11, 4.4), (11, 5.5), (11, 6.6)") // p0.b: [1, 2, 3], p1.b: [1, 2, 3, 4, 5, 6] + checkNDV("tdouble", 6, 3, 6) + tk.MustExec("insert into tdouble values (1, 4.4), (1, 5.5), (1, 6.6), (1, 7.7), (1, 8.8)") // p0.b: [1, 2, 3, 4, 5, 6, 7, 8], p1.b: [1, 2, 3, 4, 5, 6] + checkNDV("tdouble", 8, 8, 6) + + // decimal + tk.MustExec("drop table if exists tdecimal") + tk.MustExec(`CREATE TABLE tdecimal ( a int, b decimal(30, 15), key(b) ) + PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (10), + PARTITION p1 VALUES LESS THAN (20))`) + tk.MustExec("insert into tdecimal values (1, 1.1), (1, 2.2), (1, 3.3)") // p0.b: [1, 2, 3], p1.b: [] + checkNDV("tdecimal", 3, 3, 0) + tk.MustExec("insert into tdecimal values (11, 1.1), (11, 2.2), (11, 3.3)") // p0.b: [1, 2, 3], p1.b: [1, 2, 3] + checkNDV("tdecimal", 3, 3, 3) + tk.MustExec("insert into tdecimal values (11, 4.4), (11, 5.5), (11, 6.6)") // p0.b: [1, 2, 3], p1.b: [1, 2, 3, 4, 5, 6] + checkNDV("tdecimal", 6, 3, 6) + tk.MustExec("insert into tdecimal values (1, 4.4), (1, 5.5), (1, 6.6), (1, 7.7), (1, 8.8)") // p0.b: [1, 2, 3, 4, 5, 6, 7, 8], p1.b: [1, 2, 3, 4, 5, 6] + checkNDV("tdecimal", 8, 8, 6) + + // string + tk.MustExec("drop table if exists tstring") + tk.MustExec(`CREATE TABLE tstring ( a int, b varchar(30), key(b) ) + PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (10), + PARTITION p1 VALUES LESS THAN (20))`) + tk.MustExec("insert into tstring values (1, '111'), (1, '222'), (1, '333')") // p0.b: [1, 2, 3], p1.b: [] + checkNDV("tstring", 3, 3, 0) + tk.MustExec("insert into tstring values (11, '111'), (11, '222'), (11, '333')") // p0.b: [1, 2, 3], p1.b: [1, 2, 3] + checkNDV("tstring", 3, 3, 3) + tk.MustExec("insert into tstring values (11, '444'), (11, '555'), (11, '666')") // p0.b: [1, 2, 3], p1.b: [1, 2, 3, 4, 5, 6] + checkNDV("tstring", 6, 3, 6) + tk.MustExec("insert into tstring values (1, '444'), (1, '555'), (1, '666'), (1, '777'), (1, '888')") // p0.b: [1, 2, 3, 4, 5, 6, 7, 8], p1.b: [1, 2, 3, 4, 5, 6] + checkNDV("tstring", 8, 8, 6) + + // datetime + tk.MustExec("drop table if exists tdatetime") + tk.MustExec(`CREATE TABLE tdatetime ( a int, b datetime, key(b) ) + PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (10), + PARTITION p1 VALUES LESS THAN (20))`) + tk.MustExec("insert into tdatetime values (1, '2001-01-01'), (1, '2002-01-01'), (1, '2003-01-01')") // p0.b: [1, 2, 3], p1.b: [] + checkNDV("tdatetime", 3, 3, 0) + tk.MustExec("insert into tdatetime values (11, '2001-01-01'), (11, '2002-01-01'), (11, '2003-01-01')") // p0.b: [1, 2, 3], p1.b: [1, 2, 3] + checkNDV("tdatetime", 3, 3, 3) + tk.MustExec("insert into tdatetime values (11, '2004-01-01'), (11, '2005-01-01'), (11, '2006-01-01')") // p0.b: [1, 2, 3], p1.b: [1, 2, 3, 4, 5, 6] + checkNDV("tdatetime", 6, 3, 6) + tk.MustExec("insert into tdatetime values (1, '2004-01-01'), (1, '2005-01-01'), (1, '2006-01-01'), (1, '2007-01-01'), (1, '2008-01-01')") // p0.b: [1, 2, 3, 4, 5, 6, 7, 8], p1.b: [1, 2, 3, 4, 5, 6] + checkNDV("tdatetime", 8, 8, 6) +} + +func TestGlobalStats(t *testing.T) { + failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("set @@session.tidb_analyze_version = 2;") + tk.MustExec(`create table t (a int, key(a)) partition by range (a) ( + partition p0 values less than (10), + partition p1 values less than (20), + partition p2 values less than (30) + );`) + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic';") + tk.MustExec("insert into t values (1), (5), (null), (11), (15), (21), (25);") + tk.MustExec("analyze table t;") + // On the table with global-stats, we use explain to query a multi-partition query. + // And we should get the result that global-stats is used instead of pseudo-stats. + tk.MustQuery("explain format = 'brief' select a from t where a > 5").Check(testkit.Rows( + "IndexReader 4.00 root partition:all index:IndexRangeScan", + "└─IndexRangeScan 4.00 cop[tikv] table:t, index:a(a) range:(5,+inf], keep order:false")) + // On the table with global-stats, we use explain to query a single-partition query. + // And we should get the result that global-stats is used instead of pseudo-stats. + tk.MustQuery("explain format = 'brief' select * from t partition(p1) where a > 15;").Check(testkit.Rows( + "IndexReader 2.00 root partition:p1 index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:t, index:a(a) range:(15,+inf], keep order:false")) + + // Even if we have global-stats, we will not use it when the switch is set to `static`. + tk.MustExec("set @@tidb_partition_prune_mode = 'static';") + tk.MustQuery("explain format = 'brief' select a from t where a > 5").Check(testkit.Rows( + "PartitionUnion 5.00 root ", + "├─IndexReader 1.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, partition:p0, index:a(a) range:(5,+inf], keep order:false", + "├─IndexReader 2.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 2.00 cop[tikv] table:t, partition:p1, index:a(a) range:(5,+inf], keep order:false", + "└─IndexReader 2.00 root index:IndexRangeScan", + " └─IndexRangeScan 2.00 cop[tikv] table:t, partition:p2, index:a(a) range:(5,+inf], keep order:false")) + + tk.MustExec("set @@tidb_partition_prune_mode = 'static';") + tk.MustExec("drop table t;") + tk.MustExec("create table t(a int, b int, key(a)) PARTITION BY HASH(a) PARTITIONS 2;") + tk.MustExec("insert into t values(1,1),(3,3),(4,4),(2,2),(5,5);") + // When we set the mode to `static`, using analyze will not report an error and will not generate global-stats. + // In addition, when using explain to view the plan of the related query, it was found that `Union` was used. + tk.MustExec("analyze table t;") + result := tk.MustQuery("show stats_meta where table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "2", result.Rows()[0][5]) + require.Equal(t, "3", result.Rows()[1][5]) + tk.MustQuery("explain format = 'brief' select a from t where a > 3;").Check(testkit.Rows( + "PartitionUnion 2.00 root ", + "├─IndexReader 1.00 root index:IndexRangeScan", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, partition:p0, index:a(a) range:(3,+inf], keep order:false", + "└─IndexReader 1.00 root index:IndexRangeScan", + " └─IndexRangeScan 1.00 cop[tikv] table:t, partition:p1, index:a(a) range:(3,+inf], keep order:false")) + + // When we turned on the switch, we found that pseudo-stats will be used in the plan instead of `Union`. + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic';") + tk.MustQuery("explain format = 'brief' select a from t where a > 3;").Check(testkit.Rows( + "IndexReader 3333.33 root partition:all index:IndexRangeScan", + "└─IndexRangeScan 3333.33 cop[tikv] table:t, index:a(a) range:(3,+inf], keep order:false, stats:pseudo")) + + // Execute analyze again without error and can generate global-stats. + // And when executing related queries, neither Union nor pseudo-stats are used. + tk.MustExec("analyze table t;") + result = tk.MustQuery("show stats_meta where table_name = 't'").Sort() + require.Len(t, result.Rows(), 3) + require.Equal(t, "5", result.Rows()[0][5]) + require.Equal(t, "2", result.Rows()[1][5]) + require.Equal(t, "3", result.Rows()[2][5]) + tk.MustQuery("explain format = 'brief' select a from t where a > 3;").Check(testkit.Rows( + "IndexReader 2.00 root partition:all index:IndexRangeScan", + "└─IndexRangeScan 2.00 cop[tikv] table:t, index:a(a) range:(3,+inf], keep order:false")) + + tk.MustExec("drop table t;") + tk.MustExec("create table t (a int, b int, c int) PARTITION BY HASH(a) PARTITIONS 2;") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic';") + tk.MustExec("create index idx_ab on t(a, b);") + tk.MustExec("insert into t values (1, 1, 1), (5, 5, 5), (11, 11, 11), (15, 15, 15), (21, 21, 21), (25, 25, 25);") + tk.MustExec("analyze table t;") + // test the indexScan + tk.MustQuery("explain format = 'brief' select b from t where a > 5 and b > 10;").Check(testkit.Rows( + "IndexReader 2.67 root partition:all index:Projection", + "└─Projection 2.67 cop[tikv] test.t.b", + " └─Selection 2.67 cop[tikv] gt(test.t.b, 10)", + " └─IndexRangeScan 4.00 cop[tikv] table:t, index:idx_ab(a, b) range:(5,+inf], keep order:false")) + // test the indexLookUp + tk.MustQuery("explain format = 'brief' select * from t use index(idx_ab) where a > 1;").Check(testkit.Rows( + "IndexLookUp 5.00 root partition:all ", + "├─IndexRangeScan(Build) 5.00 cop[tikv] table:t, index:idx_ab(a, b) range:(1,+inf], keep order:false", + "└─TableRowIDScan(Probe) 5.00 cop[tikv] table:t keep order:false")) + // test the tableScan + tk.MustQuery("explain format = 'brief' select * from t;").Check(testkit.Rows( + "TableReader 6.00 root partition:all data:TableFullScan", + "└─TableFullScan 6.00 cop[tikv] table:t keep order:false")) +} + +func TestGlobalIndexStatistics(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + h := dom.StatsHandle() + originLease := h.Lease() + defer h.SetLease(originLease) + h.SetLease(time.Millisecond) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_enable_global_index=true") + defer func() { + tk.MustExec("set tidb_enable_global_index=default") + }() + + for i, version := range []string{"1", "2"} { + tk.MustExec("set @@session.tidb_analyze_version = " + version) + + // analyze table t + tk.MustExec("drop table if exists t") + if i != 0 { + require.Nil(t, h.HandleDDLEvent(<-h.DDLEventCh())) + } + tk.MustExec("CREATE TABLE t ( a int, b int, c int default 0, key(a) )" + + "PARTITION BY RANGE (a) (" + + "PARTITION p0 VALUES LESS THAN (10)," + + "PARTITION p1 VALUES LESS THAN (20)," + + "PARTITION p2 VALUES LESS THAN (30)," + + "PARTITION p3 VALUES LESS THAN (40))") + require.Nil(t, h.HandleDDLEvent(<-h.DDLEventCh())) + tk.MustExec("insert into t(a,b) values (1,1), (2,2), (3,3), (15,15), (25,25), (35,35)") + tk.MustExec("ALTER TABLE t ADD UNIQUE INDEX idx(b) GLOBAL") + require.Nil(t, h.DumpStatsDeltaToKV(true)) + tk.MustExec("analyze table t") + require.Nil(t, h.Update(context.Background(), dom.InfoSchema())) + tk.MustQuery("SELECT b FROM t use index(idx) WHERE b < 16 ORDER BY b"). + Check(testkit.Rows("1", "2", "3", "15")) + tk.MustQuery("EXPLAIN format='brief' SELECT b FROM t use index(idx) WHERE b < 16 ORDER BY b"). + Check(testkit.Rows("IndexReader 4.00 root partition:all index:IndexRangeScan", + "└─IndexRangeScan 4.00 cop[tikv] table:t, index:idx(b) range:[-inf,16), keep order:true")) + + // analyze table t index idx + tk.MustExec("drop table if exists t") + require.Nil(t, h.HandleDDLEvent(<-h.DDLEventCh())) + tk.MustExec("CREATE TABLE t ( a int, b int, c int default 0, primary key(b, a) clustered)" + + "PARTITION BY RANGE (a) (" + + "PARTITION p0 VALUES LESS THAN (10)," + + "PARTITION p1 VALUES LESS THAN (20)," + + "PARTITION p2 VALUES LESS THAN (30)," + + "PARTITION p3 VALUES LESS THAN (40));") + require.Nil(t, h.HandleDDLEvent(<-h.DDLEventCh())) + tk.MustExec("insert into t(a,b) values (1,1), (2,2), (3,3), (15,15), (25,25), (35,35)") + tk.MustExec("ALTER TABLE t ADD UNIQUE INDEX idx(b) GLOBAL") + require.Nil(t, h.DumpStatsDeltaToKV(true)) + tk.MustExec("analyze table t index idx") + require.Nil(t, h.Update(context.Background(), dom.InfoSchema())) + rows := tk.MustQuery("EXPLAIN SELECT b FROM t use index(idx) WHERE b < 16 ORDER BY b;").Rows() + require.Equal(t, "4.00", rows[0][1]) + + // analyze table t index + tk.MustExec("drop table if exists t") + require.Nil(t, h.HandleDDLEvent(<-h.DDLEventCh())) + tk.MustExec("CREATE TABLE t ( a int, b int, c int default 0, primary key(b, a) clustered )" + + "PARTITION BY RANGE (a) (" + + "PARTITION p0 VALUES LESS THAN (10)," + + "PARTITION p1 VALUES LESS THAN (20)," + + "PARTITION p2 VALUES LESS THAN (30)," + + "PARTITION p3 VALUES LESS THAN (40));") + require.Nil(t, h.HandleDDLEvent(<-h.DDLEventCh())) + tk.MustExec("insert into t(a,b) values (1,1), (2,2), (3,3), (15,15), (25,25), (35,35)") + tk.MustExec("ALTER TABLE t ADD UNIQUE INDEX idx(b) GLOBAL") + require.Nil(t, h.DumpStatsDeltaToKV(true)) + tk.MustExec("analyze table t index") + require.Nil(t, h.Update(context.Background(), dom.InfoSchema())) + tk.MustQuery("EXPLAIN format='brief' SELECT b FROM t use index(idx) WHERE b < 16 ORDER BY b;"). + Check(testkit.Rows("IndexReader 4.00 root partition:all index:IndexRangeScan", + "└─IndexRangeScan 4.00 cop[tikv] table:t, index:idx(b) range:[-inf,16), keep order:true")) + } +} + +func TestIssues24349(t *testing.T) { + store := testkit.CreateMockStore(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("set @@tidb_partition_prune_mode='dynamic'") + testKit.MustExec("set @@tidb_analyze_version=2") + defer testKit.MustExec("set @@tidb_analyze_version=1") + defer testKit.MustExec("set @@tidb_partition_prune_mode='static'") + testIssues24349(t, testKit, store) +} + +func TestIssues24349WithConcurrency(t *testing.T) { + store := testkit.CreateMockStore(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("set @@tidb_partition_prune_mode='dynamic'") + testKit.MustExec("set @@tidb_analyze_version=2") + testKit.MustExec("set global tidb_merge_partition_stats_concurrency=2") + defer testKit.MustExec("set @@tidb_analyze_version=1") + defer testKit.MustExec("set @@tidb_partition_prune_mode='static'") + defer testKit.MustExec("set global tidb_merge_partition_stats_concurrency=1") + testIssues24349(t, testKit, store) +} + +func TestGlobalStatsAndSQLBinding(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_merge_partition_stats_concurrency=1") + testGlobalStatsAndSQLBinding(tk) +} + +func TestGlobalStatsAndSQLBindingWithConcurrency(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_merge_partition_stats_concurrency=2") + testGlobalStatsAndSQLBinding(tk) +} diff --git a/pkg/statistics/handle/handletest/handle_test.go b/pkg/statistics/handle/handletest/handle_test.go new file mode 100644 index 0000000000000..5c83c9e23a8c5 --- /dev/null +++ b/pkg/statistics/handle/handletest/handle_test.go @@ -0,0 +1,1481 @@ +// Copyright 2017 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 handletest + +import ( + "context" + "fmt" + "strconv" + "strings" + "testing" + "time" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/cardinality" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/statistics" + "github.com/pingcap/tidb/pkg/statistics/handle" + "github.com/pingcap/tidb/pkg/statistics/handle/util" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/testkit/analyzehelper" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/collate" + "github.com/pingcap/tidb/pkg/util/mock" + "github.com/pingcap/tidb/pkg/util/ranger" + "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/oracle" +) + +func TestEmptyTable(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("create table t (c1 int, c2 int, key cc1(c1), key cc2(c2))") + testKit.MustExec("analyze table t") + do := dom + is := do.InfoSchema() + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tableInfo := tbl.Meta() + statsTbl := do.StatsHandle().GetTableStats(tableInfo) + count := cardinality.ColumnGreaterRowCount(mock.NewContext(), statsTbl, types.NewDatum(1), tableInfo.Columns[0].ID) + require.Equal(t, 0.0, count) +} + +func TestColumnIDs(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("create table t (c1 int, c2 int)") + testKit.MustExec("insert into t values(1, 2)") + testKit.MustExec("analyze table t all columns") + do := dom + is := do.InfoSchema() + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tableInfo := tbl.Meta() + statsTbl := do.StatsHandle().GetTableStats(tableInfo) + sctx := mock.NewContext() + ran := &ranger.Range{ + LowVal: []types.Datum{types.MinNotNullDatum()}, + HighVal: []types.Datum{types.NewIntDatum(2)}, + LowExclude: false, + HighExclude: true, + Collators: collate.GetBinaryCollatorSlice(1), + } + count, err := cardinality.GetRowCountByColumnRanges(sctx, &statsTbl.HistColl, tableInfo.Columns[0].ID, []*ranger.Range{ran}) + require.NoError(t, err) + require.Equal(t, float64(1), count) + + // Drop a column and the offset changed, + testKit.MustExec("alter table t drop column c1") + is = do.InfoSchema() + do.StatsHandle().Clear() + err = do.StatsHandle().Update(context.Background(), is) + require.NoError(t, err) + tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tableInfo = tbl.Meta() + statsTbl = do.StatsHandle().GetTableStats(tableInfo) + // At that time, we should get c2's stats instead of c1's. + count, err = cardinality.GetRowCountByColumnRanges(sctx, &statsTbl.HistColl, tableInfo.Columns[0].ID, []*ranger.Range{ran}) + require.NoError(t, err) + require.Equal(t, 1.0, count) +} + +func TestDurationToTS(t *testing.T) { + tests := []time.Duration{time.Millisecond, time.Second, time.Minute, time.Hour} + for _, test := range tests { + ts := util.DurationToTS(test) + require.Equal(t, int64(test), oracle.ExtractPhysical(ts)*int64(time.Millisecond)) + } +} + +func TestVersion(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit2 := testkit.NewTestKit(t, store) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("create table t1 (c1 int, c2 int)") + testKit.MustExec("analyze table t1 all columns") + do := dom + is := do.InfoSchema() + tbl1, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + require.NoError(t, err) + tableInfo1 := tbl1.Meta() + h, err := handle.NewHandle(testKit.Session(), testKit2.Session(), time.Millisecond, do.SysSessionPool(), do.SysProcTracker(), do.NextConnID, do.ReleaseConnID) + defer func() { + h.Close() + }() + require.NoError(t, err) + unit := oracle.ComposeTS(1, 0) + testKit.MustExec("update mysql.stats_meta set version = ? where table_id = ?", 2*unit, tableInfo1.ID) + + require.NoError(t, h.Update(context.Background(), is)) + require.Equal(t, 2*unit, h.MaxTableStatsVersion()) + statsTbl1 := h.GetTableStats(tableInfo1) + require.False(t, statsTbl1.Pseudo) + + testKit.MustExec("create table t2 (c1 int, c2 int)") + testKit.MustExec("analyze table t2 all columns") + is = do.InfoSchema() + tbl2, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + require.NoError(t, err) + tableInfo2 := tbl2.Meta() + // A smaller version write, and we can still read it. + testKit.MustExec("update mysql.stats_meta set version = ? where table_id = ?", unit, tableInfo2.ID) + require.NoError(t, h.Update(context.Background(), is)) + require.Equal(t, 2*unit, h.MaxTableStatsVersion()) + statsTbl2 := h.GetTableStats(tableInfo2) + require.False(t, statsTbl2.Pseudo) + + testKit.MustExec("insert t1 values(1,2)") + testKit.MustExec("analyze table t1") + offset := 3 * unit + testKit.MustExec("update mysql.stats_meta set version = ? where table_id = ?", offset+4, tableInfo1.ID) + require.NoError(t, h.Update(context.Background(), is)) + require.Equal(t, offset+uint64(4), h.MaxTableStatsVersion()) + statsTbl1 = h.GetTableStats(tableInfo1) + require.Equal(t, int64(1), statsTbl1.RealtimeCount) + + testKit.MustExec("insert t2 values(1,2)") + testKit.MustExec("analyze table t2") + // A smaller version write, and we can still read it. + testKit.MustExec("update mysql.stats_meta set version = ? where table_id = ?", offset+3, tableInfo2.ID) + require.NoError(t, h.Update(context.Background(), is)) + require.Equal(t, offset+uint64(4), h.MaxTableStatsVersion()) + statsTbl2 = h.GetTableStats(tableInfo2) + require.Equal(t, int64(1), statsTbl2.RealtimeCount) + + testKit.MustExec("insert t2 values(1,2)") + testKit.MustExec("analyze table t2") + // A smaller version write, and we cannot read it. Because at this time, lastThree Version is 4. + testKit.MustExec("update mysql.stats_meta set version = 1 where table_id = ?", tableInfo2.ID) + require.NoError(t, h.Update(context.Background(), is)) + require.Equal(t, offset+uint64(4), h.MaxTableStatsVersion()) + statsTbl2 = h.GetTableStats(tableInfo2) + require.Equal(t, int64(1), statsTbl2.RealtimeCount) + + // We add an index and analyze it, but DDL doesn't load. + testKit.MustExec("alter table t2 add column c3 int") + testKit.MustExec("analyze table t2 all columns") + // load it with old schema. + require.NoError(t, h.Update(context.Background(), is)) + statsTbl2 = h.GetTableStats(tableInfo2) + require.False(t, statsTbl2.Pseudo) + require.Nil(t, statsTbl2.GetCol(int64(3))) + // Next time DDL updated. + is = do.InfoSchema() + require.NoError(t, h.Update(context.Background(), is)) + statsTbl2 = h.GetTableStats(tableInfo2) + require.False(t, statsTbl2.Pseudo) + require.Nil(t, statsTbl2.GetCol(int64(3))) + tbl2, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + require.NoError(t, err) + tableInfo2 = tbl2.Meta() + statsTbl2, err = h.TableStatsFromStorage(tableInfo2, tableInfo2.ID, true, 0) + require.NoError(t, err) + require.NotNil(t, statsTbl2.GetCol(int64(3))) +} + +func TestLoadHist(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("create table t (c1 varchar(12), c2 char(12))") + do := dom + h := do.StatsHandle() + err := h.HandleDDLEvent(<-h.DDLEventCh()) + require.NoError(t, err) + rowCount := 10 + for i := 0; i < rowCount; i++ { + testKit.MustExec("insert into t values('a','ddd')") + } + testKit.MustExec("analyze table t") + is := do.InfoSchema() + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tableInfo := tbl.Meta() + oldStatsTbl := h.GetTableStats(tableInfo) + for i := 0; i < rowCount; i++ { + testKit.MustExec("insert into t values('bb','sdfga')") + } + require.NoError(t, h.DumpStatsDeltaToKV(true)) + err = h.Update(context.Background(), do.InfoSchema()) + require.NoError(t, err) + newStatsTbl := h.GetTableStats(tableInfo) + // The stats table is updated. + require.False(t, oldStatsTbl == newStatsTbl) + // Only the TotColSize of histograms is updated. + oldStatsTbl.ForEachColumnImmutable(func(id int64, hist *statistics.Column) bool { + require.Less(t, hist.TotColSize, newStatsTbl.GetCol(id).TotColSize) + + temp := hist.TotColSize + hist.TotColSize = newStatsTbl.GetCol(id).TotColSize + require.True(t, statistics.HistogramEqual(&hist.Histogram, &newStatsTbl.GetCol(id).Histogram, false)) + hist.TotColSize = temp + + require.True(t, hist.CMSketch.Equal(newStatsTbl.GetCol(id).CMSketch)) + require.Equal(t, newStatsTbl.GetCol(id).Info, hist.Info) + return false + }) + // Add column c3, we only update c3. + testKit.MustExec("alter table t add column c3 int") + err = h.HandleDDLEvent(<-h.DDLEventCh()) + require.NoError(t, err) + is = do.InfoSchema() + tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tableInfo = tbl.Meta() + require.NoError(t, h.Update(context.Background(), is)) + newStatsTbl2 := h.GetTableStats(tableInfo) + require.False(t, newStatsTbl2 == newStatsTbl) + // The histograms is not updated. + newStatsTbl.ForEachColumnImmutable(func(id int64, hist *statistics.Column) bool { + require.Equal(t, newStatsTbl2.GetCol(id), hist) + return false + }) + require.Greater(t, newStatsTbl2.GetCol(3).LastUpdateVersion, newStatsTbl2.GetCol(1).LastUpdateVersion) +} + +func TestCorrelation(t *testing.T) { + store := testkit.CreateMockStore(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("create table t(c1 int primary key, c2 int)") + testKit.MustExec("select * from t where c1 > 10 and c2 > 10") + testKit.MustExec("insert into t values(1,1),(3,12),(4,20),(2,7),(5,21)") + testKit.MustExec("set @@session.tidb_analyze_version=1") + testKit.MustExec("analyze table t") + result := testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "0", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) + testKit.MustExec("set @@session.tidb_analyze_version=2") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) + testKit.MustExec("insert into t values(8,18)") + testKit.MustExec("set @@session.tidb_analyze_version=1") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "0", result.Rows()[0][9]) + require.Equal(t, "0.8285714285714286", result.Rows()[1][9]) + testKit.MustExec("set @@session.tidb_analyze_version=2") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "0.8285714285714286", result.Rows()[1][9]) + + testKit.MustExec("truncate table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 0) + testKit.MustExec("insert into t values(1,21),(3,12),(4,7),(2,20),(5,1)") + testKit.MustExec("set @@session.tidb_analyze_version=1") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "0", result.Rows()[0][9]) + require.Equal(t, "-1", result.Rows()[1][9]) + testKit.MustExec("set @@session.tidb_analyze_version=2") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "-1", result.Rows()[1][9]) + testKit.MustExec("insert into t values(8,4)") + testKit.MustExec("set @@session.tidb_analyze_version=1") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "0", result.Rows()[0][9]) + require.Equal(t, "-0.9428571428571428", result.Rows()[1][9]) + testKit.MustExec("set @@session.tidb_analyze_version=2") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "-0.9428571428571428", result.Rows()[1][9]) + + testKit.MustExec("truncate table t") + testKit.MustExec("insert into t values (1,1),(2,1),(3,1),(4,1),(5,1),(6,1),(7,1),(8,1),(9,1),(10,1),(11,1),(12,1),(13,1),(14,1),(15,1),(16,1),(17,1),(18,1),(19,1),(20,2),(21,2),(22,2),(23,2),(24,2),(25,2)") + testKit.MustExec("set @@session.tidb_analyze_version=1") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "0", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) + testKit.MustExec("set @@session.tidb_analyze_version=2") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) + + testKit.MustExec("drop table t") + testKit.MustExec("create table t(c1 int, c2 int)") + testKit.MustExec("insert into t values(1,1),(2,7),(3,12),(4,20),(5,21),(8,18)") + testKit.MustExec("set @@session.tidb_analyze_version=1") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "0.8285714285714286", result.Rows()[1][9]) + testKit.MustExec("set @@session.tidb_analyze_version=2") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "0.8285714285714286", result.Rows()[1][9]) + + testKit.MustExec("truncate table t") + testKit.MustExec("insert into t values(1,1),(2,7),(3,12),(8,18),(4,20),(5,21)") + testKit.MustExec("set @@session.tidb_analyze_version=1") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "0.8285714285714286", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) + testKit.MustExec("set @@session.tidb_analyze_version=2") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "0.8285714285714286", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) + + testKit.MustExec("drop table t") + testKit.MustExec("create table t(c1 int primary key, c2 int, c3 int, key idx_c2(c2))") + testKit.MustExec("insert into t values(1,1,1),(2,2,2),(3,3,3)") + testKit.MustExec("set @@session.tidb_analyze_version=1") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't' and Is_index = 0").Sort() + require.Len(t, result.Rows(), 3) + require.Equal(t, "0", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) + require.Equal(t, "1", result.Rows()[2][9]) + result = testKit.MustQuery("show stats_histograms where Table_name = 't' and Is_index = 1").Sort() + require.Len(t, result.Rows(), 1) + require.Equal(t, "0", result.Rows()[0][9]) + testKit.MustExec("set @@tidb_analyze_version=2") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't' and Is_index = 0").Sort() + require.Len(t, result.Rows(), 3) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) + require.Equal(t, "1", result.Rows()[2][9]) + result = testKit.MustQuery("show stats_histograms where Table_name = 't' and Is_index = 1").Sort() + require.Len(t, result.Rows(), 1) + require.Equal(t, "0", result.Rows()[0][9]) +} + +func TestMergeGlobalTopN(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t;") + tk.MustExec("set @@session.tidb_analyze_version=2;") + tk.MustExec("set @@session.tidb_partition_prune_mode='dynamic';") + tk.MustExec(`create table t (a int, b int, key(b)) partition by range (a) ( + partition p0 values less than (10), + partition p1 values less than (20) + );`) + tk.MustExec("insert into t values(1, 1), (1, 1), (1, 1), (1, 1), (2, 2), (2, 2), (3, 3), (3, 3), (3, 3), " + + "(11, 11), (11, 11), (11, 11), (12, 12), (12, 12), (12, 12), (13, 3), (13, 3);") + tk.MustExec("analyze table t with 2 topn;") + // The top2 values in partition p0 are 1(count = 4) and 3(count = 3). + tk.MustQuery("show stats_topn where table_name = 't' and column_name = 'b' and partition_name = 'p0';").Check(testkit.Rows( + ("test t p0 b 0 1 4"), + ("test t p0 b 0 3 3"), + ("test t p0 b 1 1 4"), + ("test t p0 b 1 3 3"))) + // The top2 values in partition p1 are 11(count = 3) and 12(count = 3). + tk.MustQuery("show stats_topn where table_name = 't' and column_name = 'b' and partition_name = 'p1';").Check(testkit.Rows( + ("test t p1 b 0 11 3"), + ("test t p1 b 0 12 3"), + ("test t p1 b 1 11 3"), + ("test t p1 b 1 12 3"))) + // The top2 values in global are 1(count = 4) and 3(count = 5). + // Notice: The value 3 does not appear in the topN structure of partition one. + // But we can still use the histogram to calculate its accurate value. + tk.MustQuery("show stats_topn where table_name = 't' and column_name = 'b' and partition_name = 'global';").Check(testkit.Rows( + ("test t global b 0 1 4"), + ("test t global b 0 3 5"), + ("test t global b 1 1 4"), + ("test t global b 1 3 5"))) +} + +func TestExtendedStatsOps(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set session tidb_enable_extended_stats = on") + tk.MustExec("use test") + tk.MustExec("create table t(a int primary key, b int, c int, d int)") + tk.MustExec("insert into t values(1,1,5,1),(2,2,4,2),(3,3,3,3),(4,4,2,4),(5,5,1,5)") + tk.MustExec("analyze table t") + err := tk.ExecToErr("alter table not_exist_db.t add stats_extended s1 correlation(b,c)") + require.Equal(t, "[schema:1146]Table 'not_exist_db.t' doesn't exist", err.Error()) + err = tk.ExecToErr("alter table not_exist_tbl add stats_extended s1 correlation(b,c)") + require.Equal(t, "[schema:1146]Table 'test.not_exist_tbl' doesn't exist", err.Error()) + err = tk.ExecToErr("alter table t add stats_extended s1 correlation(b,e)") + require.Equal(t, "[schema:1054]Unknown column 'e' in 't'", err.Error()) + tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") + tk.MustQuery("show warnings").Check(testkit.Rows( + "Warning 1105 No need to create correlation statistics on the integer primary key column", + )) + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows()) + err = tk.ExecToErr("alter table t add stats_extended s1 correlation(b,c,d)") + require.Equal(t, "Only support Correlation and Dependency statistics types on 2 columns", err.Error()) + + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows()) + tk.MustExec("alter table t add stats_extended s1 correlation(b,c)") + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows( + "2 [2,3] 0", + )) + do := dom + is := do.InfoSchema() + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tableInfo := tbl.Meta() + err = do.StatsHandle().Update(context.Background(), is) + require.NoError(t, err) + statsTbl := do.StatsHandle().GetTableStats(tableInfo) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 0) + + tk.MustExec("update mysql.stats_extended set status = 1 where name = 's1'") + do.StatsHandle().Clear() + err = do.StatsHandle().Update(context.Background(), is) + require.NoError(t, err) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 1) + + tk.MustExec("alter table t drop stats_extended s1") + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows( + "2 [2,3] 2", + )) + err = do.StatsHandle().Update(context.Background(), is) + require.NoError(t, err) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 0) +} + +func TestAdminReloadStatistics1(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set session tidb_enable_extended_stats = on") + tk.MustExec("use test") + tk.MustExec("create table t(a int primary key, b int, c int, d int)") + tk.MustExec("insert into t values(1,1,5,1),(2,2,4,2),(3,3,3,3),(4,4,2,4),(5,5,1,5)") + tk.MustExec("analyze table t") + tk.MustExec("alter table t add stats_extended s1 correlation(b,c)") + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows( + "2 [2,3] 0", + )) + do := dom + is := do.InfoSchema() + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tableInfo := tbl.Meta() + err = do.StatsHandle().Update(context.Background(), is) + require.NoError(t, err) + statsTbl := do.StatsHandle().GetTableStats(tableInfo) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 0) + + tk.MustExec("update mysql.stats_extended set status = 1 where name = 's1'") + do.StatsHandle().Clear() + err = do.StatsHandle().Update(context.Background(), is) + require.NoError(t, err) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 1) + + tk.MustExec("delete from mysql.stats_extended where name = 's1'") + err = do.StatsHandle().Update(context.Background(), is) + require.NoError(t, err) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 1) + + tk.MustExec("admin reload stats_extended") + statsTbl = do.StatsHandle().GetTableStats(tableInfo) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 0) +} + +func TestAdminReloadStatistics2(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set session tidb_enable_extended_stats = on") + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t values(1,1),(2,2),(3,3)") + tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") + tk.MustExec("analyze table t") + tk.MustQuery("select stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows( + "1.000000 1", + )) + rows := tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() + require.Len(t, rows, 1) + + tk.MustExec("delete from mysql.stats_extended where name = 's1'") + is := dom.InfoSchema() + dom.StatsHandle().Update(context.Background(), is) + tk.MustQuery("select stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows()) + rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() + require.Len(t, rows, 1) + + tk.MustExec("admin reload stats_extended") + tk.MustQuery("select stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows()) + rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() + require.Len(t, rows, 0) +} + +func TestCorrelationStatsCompute(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set session tidb_enable_extended_stats = on") + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int, c int, index idx(a, b, c))") + tk.MustExec("insert into t values(1,1,5),(2,2,4),(3,3,3),(4,4,2),(5,5,1)") + tk.MustExec("analyze table t") + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Check(testkit.Rows()) + tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") + tk.MustExec("alter table t add stats_extended s2 correlation(a,c)") + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( + "2 [1,2] 0", + "2 [1,3] 0", + )) + do := dom + is := do.InfoSchema() + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tableInfo := tbl.Meta() + err = do.StatsHandle().Update(context.Background(), is) + require.NoError(t, err) + statsTbl := do.StatsHandle().GetTableStats(tableInfo) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 0) + + tk.MustExec("analyze table t") + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( + "2 [1,2] 1.000000 1", + "2 [1,3] -1.000000 1", + )) + tk.MustExec("set @@session.tidb_analyze_version=2") + tk.MustExec("analyze table t") + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( + "2 [1,2] 1.000000 1", + "2 [1,3] -1.000000 1", + )) + err = do.StatsHandle().Update(context.Background(), is) + require.NoError(t, err) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 2) + foundS1, foundS2 := false, false + for name, item := range statsTbl.ExtendedStats.Stats { + switch name { + case "s1": + foundS1 = true + require.Equal(t, float64(1), item.ScalarVals) + case "s2": + foundS2 = true + require.Equal(t, float64(-1), item.ScalarVals) + default: + require.FailNow(t, "Unexpected extended stats in cache") + } + } + require.True(t, foundS1 && foundS2) + + // Check that table with NULLs won't cause panic + tk.MustExec("delete from t") + tk.MustExec("insert into t values(1,null,2), (2,null,null)") + tk.MustExec("set @@session.tidb_analyze_version=1") + tk.MustExec("analyze table t") + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( + "2 [1,2] 0.000000 1", + "2 [1,3] 1.000000 1", + )) + tk.MustExec("set @@session.tidb_analyze_version=2") + tk.MustExec("analyze table t") + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( + "2 [1,2] 0.000000 1", + "2 [1,3] 1.000000 1", + )) + tk.MustExec("insert into t values(3,3,3)") + tk.MustExec("set @@session.tidb_analyze_version=1") + tk.MustExec("analyze table t") + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( + "2 [1,2] 1.000000 1", + "2 [1,3] 1.000000 1", + )) + tk.MustExec("set @@session.tidb_analyze_version=2") + tk.MustExec("analyze table t") + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( + "2 [1,2] 1.000000 1", + "2 [1,3] 1.000000 1", + )) +} + +func TestSyncStatsExtendedRemoval(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set session tidb_enable_extended_stats = on") + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t values(1,1),(2,2),(3,3)") + tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") + tk.MustExec("analyze table t") + do := dom + is := do.InfoSchema() + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tableInfo := tbl.Meta() + statsTbl := do.StatsHandle().GetTableStats(tableInfo) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 1) + item := statsTbl.ExtendedStats.Stats["s1"] + require.NotNil(t, item) + result := tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") + require.Len(t, result.Rows(), 1) + + tk.MustExec("alter table t drop stats_extended s1") + statsTbl = do.StatsHandle().GetTableStats(tableInfo) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 0) + result = tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") + require.Len(t, result.Rows(), 0) +} + +func TestStaticPartitionPruneMode(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Static) + "'") + tk.MustExec("use test") + tk.MustExec(`create table t (a int, key(a)) partition by range(a) + (partition p0 values less than (10), + partition p1 values less than (22))`) + tk.MustExec(`insert into t values (1), (2), (3), (10), (11)`) + tk.MustExec(`analyze table t`) + require.True(t, tk.MustNoGlobalStats("t")) + tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Dynamic) + "'") + require.True(t, tk.MustNoGlobalStats("t")) + + tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Static) + "'") + tk.MustExec(`insert into t values (4), (5), (6)`) + tk.MustExec(`analyze table t partition p0`) + require.True(t, tk.MustNoGlobalStats("t")) + tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Dynamic) + "'") + require.True(t, tk.MustNoGlobalStats("t")) + tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Static) + "'") +} + +func TestMergeIdxHist(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Dynamic) + "'") + defer tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Static) + "'") + tk.MustExec("use test") + tk.MustExec(` + create table t (a int, key(a)) + partition by range (a) ( + partition p0 values less than (10), + partition p1 values less than (20))`) + tk.MustExec("set @@tidb_analyze_version=2") + defer tk.MustExec("set @@tidb_analyze_version=1") + tk.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (6), (null), (11), (12), (13), (14), (15), (16), (17), (18), (19), (19)") + + tk.MustExec("analyze table t with 2 topn, 2 buckets") + rows := tk.MustQuery("show stats_buckets where partition_name like 'global'") + require.Len(t, rows.Rows(), 4) +} + +func TestPartitionPruneModeSessionVariable(t *testing.T) { + failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") + + store := testkit.CreateMockStore(t) + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk1.MustExec("set tidb_cost_model_version=1") + tk1.MustExec("set @@tidb_partition_prune_mode = '" + string(variable.Dynamic) + "'") + tk1.MustExec(`set @@tidb_analyze_version=2`) + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + tk2.MustExec("set tidb_cost_model_version=1") + tk2.MustExec("set @@tidb_partition_prune_mode = '" + string(variable.Static) + "'") + tk2.MustExec(`set @@tidb_analyze_version=2`) + + tk1.MustExec(`create table t (a int, key(a)) partition by range(a) + (partition p0 values less than (10), + partition p1 values less than (22))`) + + tk1.MustQuery("explain format = 'brief' select * from t").Check(testkit.Rows( + "TableReader 10000.00 root partition:all data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + )) + tk2.MustQuery("explain format = 'brief' select * from t").Check(testkit.Rows( + "PartitionUnion 20000.00 root ", + "├─TableReader 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + "└─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + )) + + tk1.MustExec(`insert into t values (1), (2), (3), (10), (11)`) + tk1.MustExec(`analyze table t with 1 topn, 2 buckets`) + tk1.MustQuery("explain format = 'brief' select * from t").Check(testkit.Rows( + "TableReader 5.00 root partition:all data:TableFullScan", + "└─TableFullScan 5.00 cop[tikv] table:t keep order:false", + )) + tk2.MustQuery("explain format = 'brief' select * from t").Check(testkit.Rows( + "PartitionUnion 5.00 root ", + "├─TableReader 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t, partition:p0 keep order:false", + "└─TableReader 2.00 root data:TableFullScan", + " └─TableFullScan 2.00 cop[tikv] table:t, partition:p1 keep order:false", + )) + + tk1.MustExec("set @@tidb_partition_prune_mode = '" + string(variable.Static) + "'") + tk1.MustQuery("explain format = 'brief' select * from t").Check(testkit.Rows( + "PartitionUnion 5.00 root ", + "├─TableReader 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t, partition:p0 keep order:false", + "└─TableReader 2.00 root data:TableFullScan", + " └─TableFullScan 2.00 cop[tikv] table:t, partition:p1 keep order:false", + )) + tk2.MustExec("set @@tidb_partition_prune_mode = '" + string(variable.Dynamic) + "'") + tk2.MustQuery("explain format = 'brief' select * from t").Check(testkit.Rows( + "TableReader 5.00 root partition:all data:TableFullScan", + "└─TableFullScan 5.00 cop[tikv] table:t keep order:false", + )) +} + +func TestRepetitiveAddDropExtendedStats(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set session tidb_enable_extended_stats = on") + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t values(1,1),(2,2),(3,3)") + tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") + tk.MustQuery("select name, status from mysql.stats_extended where name = 's1'").Sort().Check(testkit.Rows( + "s1 0", + )) + result := tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") + require.Len(t, result.Rows(), 0) + tk.MustExec("analyze table t") + tk.MustQuery("select name, status from mysql.stats_extended where name = 's1'").Sort().Check(testkit.Rows( + "s1 1", + )) + result = tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") + require.Len(t, result.Rows(), 1) + tk.MustExec("alter table t drop stats_extended s1") + tk.MustQuery("select name, status from mysql.stats_extended where name = 's1'").Sort().Check(testkit.Rows( + "s1 2", + )) + result = tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") + require.Len(t, result.Rows(), 0) + tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") + tk.MustQuery("select name, status from mysql.stats_extended where name = 's1'").Sort().Check(testkit.Rows( + "s1 0", + )) + result = tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") + require.Len(t, result.Rows(), 0) + tk.MustExec("analyze table t") + tk.MustQuery("select name, status from mysql.stats_extended where name = 's1'").Sort().Check(testkit.Rows( + "s1 1", + )) + result = tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") + require.Len(t, result.Rows(), 1) +} + +func TestDuplicateFMSketch(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + defer tk.MustExec("set @@tidb_partition_prune_mode='static'") + tk.MustExec("create table t(a int, b int, c int) partition by hash(a) partitions 3") + tk.MustExec("insert into t values (1, 1, 1)") + analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a", "b", "c") + tk.MustExec("analyze table t") + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("9")) + tk.MustExec("analyze table t") + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("9")) + + tk.MustExec("alter table t drop column b") + require.NoError(t, dom.StatsHandle().GCStats(dom.InfoSchema(), time.Duration(0))) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("6")) +} + +func TestIndexFMSketch(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@session.tidb_analyze_version = 1") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, c int, index ia(a), index ibc(b, c)) partition by hash(a) partitions 3") + tk.MustExec("insert into t values (1, 1, 1)") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + defer tk.MustExec("set @@tidb_partition_prune_mode='static'") + tk.MustExec("analyze table t index ia") + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("3")) + tk.MustExec("analyze table t index ibc") + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("6")) + tk.MustExec("analyze table t") + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("15")) + tk.MustExec("drop table if exists t") + require.NoError(t, dom.StatsHandle().GCStats(dom.InfoSchema(), 0)) + + // clustered index + tk.MustExec("drop table if exists t") + tk.MustExec("set @@tidb_enable_clustered_index=ON") + tk.MustExec("create table t (a datetime, b datetime, primary key (a)) partition by hash(year(a)) partitions 3") + tk.MustExec("insert into t values ('2000-01-01', '2000-01-01')") + tk.MustExec("analyze table t") + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("6")) + tk.MustExec("drop table if exists t") + require.NoError(t, dom.StatsHandle().GCStats(dom.InfoSchema(), 0)) + + // test NDV + checkNDV := func(rows, ndv int) { + tk.MustExec("analyze table t") + rs := tk.MustQuery("select value from mysql.stats_fm_sketch").Rows() + require.Len(t, rs, rows) + for i := range rs { + fm, err := statistics.DecodeFMSketch([]byte(rs[i][0].(string))) + require.NoError(t, err) + require.Equal(t, int64(ndv), fm.NDV()) + } + } + + tk.MustExec("set @@tidb_enable_clustered_index=OFF") + tk.MustExec("create table t(a int, key(a)) partition by hash(a) partitions 3") + tk.MustExec("insert into t values (1), (2), (2), (3)") + checkNDV(6, 1) + tk.MustExec("insert into t values (4), (5), (6)") + checkNDV(6, 2) + tk.MustExec("insert into t values (2), (5)") + checkNDV(6, 2) + tk.MustExec("drop table if exists t") + require.NoError(t, dom.StatsHandle().GCStats(dom.InfoSchema(), 0)) + + // clustered index + tk.MustExec("set @@tidb_enable_clustered_index=ON") + tk.MustExec("create table t (a datetime, b datetime, primary key (a)) partition by hash(year(a)) partitions 3") + tk.MustExec("insert into t values ('2000-01-01', '2001-01-01'), ('2001-01-01', '2001-01-01'), ('2002-01-01', '2001-01-01')") + checkNDV(6, 1) + tk.MustExec("insert into t values ('1999-01-01', '1998-01-01'), ('1997-01-02', '1999-01-02'), ('1998-01-03', '1999-01-03')") + checkNDV(6, 2) +} + +func TestShowExtendedStats4DropColumn(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set session tidb_enable_extended_stats = on") + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int, c int)") + tk.MustExec("insert into t values(1,1,1),(2,2,2),(3,3,3)") + tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") + tk.MustExec("alter table t add stats_extended s2 correlation(b,c)") + tk.MustExec("analyze table t") + rows := tk.MustQuery("show stats_extended").Sort().Rows() + require.Len(t, rows, 2) + require.Equal(t, "s1", rows[0][2]) + require.Equal(t, "[a,b]", rows[0][3]) + require.Equal(t, "s2", rows[1][2]) + require.Equal(t, "[b,c]", rows[1][3]) + + tk.MustExec("alter table t drop column b") + rows = tk.MustQuery("show stats_extended").Rows() + require.Len(t, rows, 0) + + // Previously registered extended stats should be invalid for re-created columns. + tk.MustExec("alter table t add column b int") + rows = tk.MustQuery("show stats_extended").Rows() + require.Len(t, rows, 0) +} + +func TestExtStatsOnReCreatedTable(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set session tidb_enable_extended_stats = on") + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t values(1,1),(2,2),(3,3)") + tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") + tk.MustExec("analyze table t") + rows := tk.MustQuery("select table_id, stats from mysql.stats_extended where name = 's1'").Rows() + require.Len(t, rows, 1) + tableID1 := rows[0][0] + require.Equal(t, "1.000000", rows[0][1]) + rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() + require.Len(t, rows, 1) + require.Equal(t, "1.000000", rows[0][5]) + + tk.MustExec("drop table t") + rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() + require.Len(t, rows, 0) + + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t values(1,3),(2,2),(3,1)") + tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") + tk.MustExec("analyze table t") + rows = tk.MustQuery("select table_id, stats from mysql.stats_extended where name = 's1' order by stats").Rows() + require.Len(t, rows, 2) + tableID2 := rows[0][0] + require.NotEqual(t, tableID1, tableID2) + require.Equal(t, tableID1, rows[1][0]) + require.Equal(t, "-1.000000", rows[0][1]) + require.Equal(t, "1.000000", rows[1][1]) + rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() + require.Len(t, rows, 1) + require.Equal(t, "-1.000000", rows[0][5]) +} + +func TestExtStatsOnReCreatedColumn(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set session tidb_enable_extended_stats = on") + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t values(1,1),(2,2),(3,3)") + tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") + tk.MustExec("analyze table t") + tk.MustQuery("select column_ids, stats from mysql.stats_extended where name = 's1'").Check(testkit.Rows( + "[1,2] 1.000000", + )) + rows := tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() + require.Len(t, rows, 1) + require.Equal(t, "[a,b]", rows[0][3]) + require.Equal(t, "1.000000", rows[0][5]) + + tk.MustExec("alter table t drop column b") + tk.MustExec("alter table t add column b int") + tk.MustQuery("select * from t").Sort().Check(testkit.Rows( + "1 ", + "2 ", + "3 ", + )) + tk.MustExec("update t set b = 3 where a = 1") + tk.MustExec("update t set b = 2 where a = 2") + tk.MustExec("update t set b = 1 where a = 3") + tk.MustQuery("select * from t").Sort().Check(testkit.Rows( + "1 3", + "2 2", + "3 1", + )) + tk.MustExec("analyze table t") + // Previous extended stats would not be collected and would not take effect anymore, it will be removed by stats GC. + tk.MustQuery("select column_ids, stats from mysql.stats_extended where name = 's1'").Check(testkit.Rows( + "[1,2] 1.000000", + )) + rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() + require.Len(t, rows, 0) +} + +func TestExtStatsOnRenamedColumn(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set session tidb_enable_extended_stats = on") + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t values(1,1),(2,2),(3,3)") + tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") + tk.MustExec("analyze table t") + tk.MustQuery("select column_ids, stats from mysql.stats_extended where name = 's1'").Check(testkit.Rows( + "[1,2] 1.000000", + )) + rows := tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() + require.Len(t, rows, 1) + require.Equal(t, "[a,b]", rows[0][3]) + require.Equal(t, "1.000000", rows[0][5]) + + tk.MustExec("alter table t rename column b to c") + tk.MustExec("update t set c = 3 where a = 1") + tk.MustExec("update t set c = 2 where a = 2") + tk.MustExec("update t set c = 1 where a = 3") + tk.MustQuery("select * from t").Sort().Check(testkit.Rows( + "1 3", + "2 2", + "3 1", + )) + tk.MustExec("analyze table t") + // Previous extended stats would still be collected and take effect. + tk.MustQuery("select column_ids, stats from mysql.stats_extended where name = 's1'").Check(testkit.Rows( + "[1,2] -1.000000", + )) + rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() + require.Len(t, rows, 1) + require.Equal(t, "[a,c]", rows[0][3]) + require.Equal(t, "-1.000000", rows[0][5]) +} + +func TestExtStatsOnModifiedColumn(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set session tidb_enable_extended_stats = on") + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t values(1,1),(2,2),(3,3)") + tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") + tk.MustExec("analyze table t") + tk.MustQuery("select column_ids, stats from mysql.stats_extended where name = 's1'").Check(testkit.Rows( + "[1,2] 1.000000", + )) + rows := tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() + require.Len(t, rows, 1) + require.Equal(t, "[a,b]", rows[0][3]) + require.Equal(t, "1.000000", rows[0][5]) + + tk.MustExec("alter table t modify column b bigint") + tk.MustExec("update t set b = 3 where a = 1") + tk.MustExec("update t set b = 2 where a = 2") + tk.MustExec("update t set b = 1 where a = 3") + tk.MustQuery("select * from t").Sort().Check(testkit.Rows( + "1 3", + "2 2", + "3 1", + )) + tk.MustExec("analyze table t") + // Previous extended stats would still be collected and take effect. + tk.MustQuery("select column_ids, stats from mysql.stats_extended where name = 's1'").Check(testkit.Rows( + "[1,2] -1.000000", + )) + rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() + require.Len(t, rows, 1) + require.Equal(t, "[a,b]", rows[0][3]) + require.Equal(t, "-1.000000", rows[0][5]) +} + +func TestCorrelationWithDefinedCollate(t *testing.T) { + store := testkit.CreateMockStore(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(a int primary key, b varchar(8) character set utf8mb4 collate utf8mb4_general_ci, c varchar(8) character set utf8mb4 collate utf8mb4_bin)") + testKit.MustExec("insert into t values(1,'aa','aa'),(2,'Cb','Cb'),(3,'CC','CC')") + analyzehelper.TriggerPredicateColumnsCollection(t, testKit, store, "t", "a", "b", "c") + testKit.MustExec("analyze table t") + testKit.MustQuery("select a from t order by b").Check(testkit.Rows( + "1", + "2", + "3", + )) + testKit.MustQuery("select a from t order by c").Check(testkit.Rows( + "3", + "2", + "1", + )) + rows := testKit.MustQuery("show stats_histograms where table_name = 't'").Sort().Rows() + require.Len(t, rows, 3) + require.Equal(t, "1", rows[1][9]) + require.Equal(t, "-1", rows[2][9]) + testKit.MustExec("set session tidb_enable_extended_stats = on") + testKit.MustExec("alter table t add stats_extended s1 correlation(b,c)") + testKit.MustExec("analyze table t") + rows = testKit.MustQuery("show stats_extended where stats_name = 's1'").Sort().Rows() + require.Len(t, rows, 1) + require.Equal(t, "[b,c]", rows[0][3]) + require.Equal(t, "-1.000000", rows[0][5]) +} + +func TestLoadHistogramWithCollate(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(a varchar(10) collate utf8mb4_unicode_ci);") + testKit.MustExec("insert into t values('abcdefghij');") + testKit.MustExec("insert into t values('abcdufghij');") + testKit.MustExec("analyze table t with 0 topn;") + do := dom + h := do.StatsHandle() + is := do.InfoSchema() + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo := tbl.Meta() + _, err = h.TableStatsFromStorage(tblInfo, tblInfo.ID, true, 0) + require.NoError(t, err) +} + +func TestStatsCacheUpdateSkip(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + do := dom + h := do.StatsHandle() + testKit.MustExec("use test") + testKit.MustExec("create table t (c1 int, c2 int)") + testKit.MustExec("insert into t values(1, 2)") + require.NoError(t, h.DumpStatsDeltaToKV(true)) + testKit.MustExec("analyze table t") + is := do.InfoSchema() + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tableInfo := tbl.Meta() + statsTbl1 := h.GetTableStats(tableInfo) + require.False(t, statsTbl1.Pseudo) + h.Update(context.Background(), is) + statsTbl2 := h.GetTableStats(tableInfo) + require.Equal(t, statsTbl2, statsTbl1) +} + +func testIncrementalModifyCountUpdateHelper(analyzeSnapshot bool) func(*testing.T) { + return func(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + if analyzeSnapshot { + tk.MustExec("set @@session.tidb_enable_analyze_snapshot = on") + } else { + tk.MustExec("set @@session.tidb_enable_analyze_snapshot = off") + } + tk.MustExec("create table t(a int)") + analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a") + tk.MustExec("set @@session.tidb_analyze_version = 2") + h := dom.StatsHandle() + err := h.HandleDDLEvent(<-h.DDLEventCh()) + require.NoError(t, err) + tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo := tbl.Meta() + tid := tblInfo.ID + + tk.MustExec("insert into t values(1),(2),(3)") + require.NoError(t, h.DumpStatsDeltaToKV(true)) + err = h.Update(context.Background(), dom.InfoSchema()) + require.NoError(t, err) + tk.MustExec("analyze table t") + tk.MustQuery(fmt.Sprintf("select count, modify_count from mysql.stats_meta where table_id = %d", tid)).Check(testkit.Rows( + "3 0", + )) + + tk.MustExec("begin") + txn, err := tk.Session().Txn(false) + require.NoError(t, err) + startTS := txn.StartTS() + tk.MustExec("commit") + + tk.MustExec("insert into t values(4),(5),(6)") + require.NoError(t, h.DumpStatsDeltaToKV(true)) + err = h.Update(context.Background(), dom.InfoSchema()) + require.NoError(t, err) + + // Simulate that the analyze would start before and finish after the second insert. + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/injectAnalyzeSnapshot", fmt.Sprintf("return(%d)", startTS))) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/injectBaseCount", "return(3)")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/injectBaseModifyCount", "return(0)")) + tk.MustExec("analyze table t") + if analyzeSnapshot { + // Check the count / modify_count changes during the analyze are not lost. + tk.MustQuery(fmt.Sprintf("select count, modify_count from mysql.stats_meta where table_id = %d", tid)).Check(testkit.Rows( + "6 3", + )) + // Check the histogram is correct for the snapshot analyze. + tk.MustQuery(fmt.Sprintf("select distinct_count from mysql.stats_histograms where table_id = %d", tid)).Check(testkit.Rows( + "3", + )) + } else { + // Since analyze use max ts to read data, it finds the row count is 6 and directly set count to 6 rather than incrementally update it. + // But it still incrementally updates modify_count. + tk.MustQuery(fmt.Sprintf("select count, modify_count from mysql.stats_meta where table_id = %d", tid)).Check(testkit.Rows( + "6 3", + )) + // Check the histogram is collected from the latest data rather than the snapshot at startTS. + tk.MustQuery(fmt.Sprintf("select distinct_count from mysql.stats_histograms where table_id = %d", tid)).Check(testkit.Rows( + "6", + )) + } + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/executor/injectAnalyzeSnapshot")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/executor/injectBaseCount")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/executor/injectBaseModifyCount")) + } +} + +func TestIncrementalModifyCountUpdate(t *testing.T) { + for _, analyzeSnapshot := range []bool{true, false} { + t.Run(fmt.Sprintf("%s-%t", t.Name(), analyzeSnapshot), testIncrementalModifyCountUpdateHelper(analyzeSnapshot)) + } +} + +func TestRecordHistoricalStatsToStorage(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b varchar(10))") + tk.MustExec("insert into t value(1, 'aaa'), (3, 'aab'), (5, 'bba'), (2, 'bbb'), (4, 'cca'), (6, 'ccc')") + // mark column stats as needed + tk.MustExec("select * from t where a = 3") + tk.MustExec("select * from t where b = 'bbb'") + tk.MustExec("alter table t add index single(a)") + tk.MustExec("alter table t add index multi(a, b)") + tk.MustExec("analyze table t with 2 topn") + + tableInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + version, err := dom.StatsHandle().RecordHistoricalStatsToStorage("t", tableInfo.Meta(), tableInfo.Meta().ID, false) + require.NoError(t, err) + + rows := tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where version = '%d'", version)).Rows() + num, _ := strconv.Atoi(rows[0][0].(string)) + require.GreaterOrEqual(t, num, 1) +} + +func TestEvictedColumnLoadedStatus(t *testing.T) { + t.Skip("skip this test because it is useless") + restore := config.RestoreFunc() + defer restore() + config.UpdateGlobal(func(conf *config.Config) { + conf.Performance.EnableStatsCacheMemQuota = true + }) + store, dom := testkit.CreateMockStoreAndDomain(t) + dom.StatsHandle().SetLease(0) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_analyze_version = 1") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("analyze table test.t") + tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.Nil(t, err) + tblStats := domain.GetDomain(tk.Session()).StatsHandle().GetTableStats(tbl.Meta()) + tblStats.ForEachColumnImmutable(func(_ int64, col *statistics.Column) bool { + require.True(t, col.IsStatsInitialized()) + return false + }) + + domain.GetDomain(tk.Session()).StatsHandle().SetStatsCacheCapacity(1) + tblStats = domain.GetDomain(tk.Session()).StatsHandle().GetTableStats(tbl.Meta()) + tblStats.ForEachColumnImmutable(func(_ int64, col *statistics.Column) bool { + require.True(t, col.IsStatsInitialized()) + return false + }) +} + +func TestUninitializedStatsStatus(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + dom.StatsHandle().SetLease(0) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, c int, index idx_a(a))") + h := dom.StatsHandle() + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + tk.MustExec("insert into t values (1,2,2), (3,4,4), (5,6,6), (7,8,8), (9,10,10)") + require.NoError(t, h.DumpStatsDeltaToKV(true)) + is := dom.InfoSchema() + require.NoError(t, h.Update(context.Background(), is)) + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo := tbl.Meta() + tblStats := h.GetTableStats(tblInfo) + tblStats.ForEachColumnImmutable(func(_ int64, col *statistics.Column) bool { + require.False(t, col.IsStatsInitialized()) + return false + }) + tblStats.ForEachIndexImmutable(func(_ int64, idx *statistics.Index) bool { + require.False(t, idx.IsStatsInitialized()) + return false + }) + tk.MustQuery("show stats_histograms where db_name = 'test' and table_name = 't'").Check(testkit.Rows()) + checkStatsPseudo := func() { + rows := tk.MustQuery("explain select * from t").Rows() + operatorInfo := rows[len(rows)-1][4].(string) + require.True(t, strings.Contains(operatorInfo, "stats:pseudo")) + } + tk.MustExec("set @@tidb_enable_pseudo_for_outdated_stats = true") + checkStatsPseudo() + tk.MustExec("set @@tidb_enable_pseudo_for_outdated_stats = false") + checkStatsPseudo() +} + +func TestIssue39336(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(` +create table t1 ( + a datetime(3) default null, + b int +) partition by range (b) ( + partition p0 values less than (1000), + partition p1 values less than (maxvalue) +)`) + tk.MustExec("set @@sql_mode=''") + tk.MustExec("set @@tidb_analyze_version=2") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + tk.MustExec(` +insert into t1 values +('1000-00-09 00:00:00.000', 1), +('1000-00-06 00:00:00.000', 1), +('1000-00-06 00:00:00.000', 1), +('2022-11-23 14:24:30.000', 1), +('2022-11-23 14:24:32.000', 1), +('2022-11-23 14:24:33.000', 1), +('2022-11-23 14:24:35.000', 1), +('2022-11-23 14:25:08.000', 1001), +('2022-11-23 14:25:09.000', 1001)`) + analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t1", "a", "b") + tk.MustExec("analyze table t1 with 0 topn") + rows := tk.MustQuery("show analyze status where job_info like 'merge global stats%'").Rows() + require.Len(t, rows, 1) + require.Equal(t, "finished", rows[0][7]) +} + +func checkAllEvicted(t *testing.T, statsTbl *statistics.Table) { + statsTbl.ForEachColumnImmutable(func(_ int64, col *statistics.Column) bool { + require.True(t, col.IsAllEvicted()) + return false + }) + statsTbl.ForEachIndexImmutable(func(_ int64, idx *statistics.Index) bool { + require.True(t, idx.IsAllEvicted()) + return false + }) +} + +func TestInitStatsLite(t *testing.T) { + oriVal := config.GetGlobalConfig().Performance.LiteInitStats + config.GetGlobalConfig().Performance.LiteInitStats = true + defer func() { + config.GetGlobalConfig().Performance.LiteInitStats = oriVal + }() + + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int, c int, primary key(a), key idxb(b), key idxc(c))") + tk.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9)") + + h := dom.StatsHandle() + // set lease > 0 to trigger on-demand stats load. + h.SetLease(time.Millisecond) + defer func() { + h.SetLease(0) + }() + + is := dom.InfoSchema() + tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo := tbl.Meta() + colBID := tblInfo.Columns[1].ID + colCID := tblInfo.Columns[2].ID + idxBID := tblInfo.Indices[0].ID + idxCID := tblInfo.Indices[1].ID + + tk.MustExec("analyze table t with 2 topn, 2 buckets") + statsTbl0 := h.GetTableStats(tblInfo) + checkAllEvicted(t, statsTbl0) + + h.Clear() + require.NoError(t, h.InitStatsLite(context.Background(), is)) + statsTbl1 := h.GetTableStats(tblInfo) + checkAllEvicted(t, statsTbl1) + { + // internal.AssertTableEqual(t, statsTbl0, statsTbl1) + // statsTbl0 is loaded when the cache has pseudo table. + // TODO: We haven't optimize the pseudo table's memory usage yet. So here the two will be different. + require.True(t, statsTbl0.ColNum() > 0) + require.True(t, statsTbl0.IdxNum() > 0) + require.True(t, statsTbl1.ColNum() == 0) + require.True(t, statsTbl1.IdxNum() == 0) + } + + // async stats load + tk.MustExec("set @@tidb_stats_load_sync_wait = 0") + tk.MustExec("explain select * from t where b > 1") + require.NoError(t, h.LoadNeededHistograms()) + statsTbl2 := h.GetTableStats(tblInfo) + colBStats1 := statsTbl2.GetCol(colBID) + colCStats := statsTbl2.GetCol(colCID) + require.True(t, colBStats1.IsFullLoad()) + idxBStats1 := statsTbl2.GetIdx(idxBID) + require.True(t, idxBStats1.IsFullLoad()) + require.True(t, colCStats.IsAllEvicted()) + + // sync stats load + tk.MustExec("set @@tidb_stats_load_sync_wait = 60000") + tk.MustExec("explain select * from t where c > 1") + statsTbl3 := h.GetTableStats(tblInfo) + colCStats1 := statsTbl3.GetCol(colCID) + require.True(t, colCStats1.IsFullLoad()) + idxCStats1 := statsTbl3.GetIdx(idxCID) + require.True(t, idxCStats1.IsFullLoad()) + + // update stats + tk.MustExec("analyze table t with 1 topn, 3 buckets") + statsTbl4 := h.GetTableStats(tblInfo) + colBStats2 := statsTbl4.GetCol(colBID) + require.True(t, colBStats2.IsFullLoad()) + require.Greater(t, colBStats2.LastUpdateVersion, colBStats1.LastUpdateVersion) + idxBStats2 := statsTbl4.GetIdx(idxBID) + require.True(t, idxBStats2.IsFullLoad()) + require.Greater(t, idxBStats2.LastUpdateVersion, idxBStats1.LastUpdateVersion) + colCStats2 := statsTbl4.GetCol(colCID) + require.True(t, colCStats2.IsFullLoad()) + require.Greater(t, colCStats2.LastUpdateVersion, colCStats1.LastUpdateVersion) + idxCStats2 := statsTbl4.GetIdx(idxCID) + require.True(t, idxCStats2.IsFullLoad()) + require.Greater(t, idxCStats2.LastUpdateVersion, idxCStats1.LastUpdateVersion) +} + +func TestSkipMissingPartitionStats(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("set @@tidb_skip_missing_partition_stats = 1") + tk.MustExec("create table t (a int, b int, c int, index idx_b(b)) partition by range (a) (partition p0 values less than (100), partition p1 values less than (200), partition p2 values less than (300))") + tk.MustExec("insert into t values (1,1,1), (2,2,2), (101,101,101), (102,102,102), (201,201,201), (202,202,202)") + analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a", "b", "c") + h := dom.StatsHandle() + require.NoError(t, h.DumpStatsDeltaToKV(true)) + tk.MustExec("analyze table t partition p0, p1") + tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo := tbl.Meta() + globalStats := h.GetTableStats(tblInfo) + require.Equal(t, 6, int(globalStats.RealtimeCount)) + require.Equal(t, 2, int(globalStats.ModifyCount)) + globalStats.ForEachColumnImmutable(func(_ int64, col *statistics.Column) bool { + require.True(t, col.IsStatsInitialized()) + return false + }) + globalStats.ForEachIndexImmutable(func(_ int64, idx *statistics.Index) bool { + require.True(t, idx.IsStatsInitialized()) + return false + }) +} diff --git a/pkg/statistics/testdata/integration_suite_out.json b/pkg/statistics/testdata/integration_suite_out.json new file mode 100644 index 0000000000000..14cdc66e90ce0 --- /dev/null +++ b/pkg/statistics/testdata/integration_suite_out.json @@ -0,0 +1,51 @@ +[ + { + "Name": "TestExpBackoffEstimation", + "Cases": [ + [ + "IndexReader_6 5.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 5.00 cop[tikv] table:exp_backoff, index:idx(a, b, c, d) range:[1,1], keep order:false" + ], + [ + "IndexReader_10 3.00 root index:Selection_9", + "└─Selection_9 3.00 cop[tikv] eq(test.exp_backoff.b, 1)", + " └─IndexFullScan_8 5.00 cop[tikv] table:exp_backoff, index:idx(a, b, c, d) keep order:false" + ], + [ + "IndexReader_10 2.00 root index:Selection_9", + "└─Selection_9 2.00 cop[tikv] eq(test.exp_backoff.c, 1)", + " └─IndexFullScan_8 5.00 cop[tikv] table:exp_backoff, index:idx(a, b, c, d) keep order:false" + ], + [ + "IndexReader_10 3.00 root index:Selection_9", + "└─Selection_9 3.00 cop[tikv] ge(test.exp_backoff.d, 3), le(test.exp_backoff.d, 5)", + " └─IndexFullScan_8 5.00 cop[tikv] table:exp_backoff, index:idx(a, b, c, d) keep order:false" + ], + [ + "IndexReader_6 1.36 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 1.36 cop[tikv] table:exp_backoff, index:idx(a, b, c, d) range:[1 1 1 3,1 1 1 5], keep order:false" + ], + [ + "IndexReader_6 1.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 1.00 cop[tikv] table:exp_backoff, index:idx(a, b, c, d) range:[1 1 1 3,1 1 1 5], keep order:false" + ] + ] + }, + { + "Name": "TestNULLOnFullSampling", + "Cases": [ + [ + "test t a 0 1 3", + "test t a 0 2 2", + "test t idx 1 1 3", + "test t idx 1 2 2" + ], + [ + "test t a 0 0 1 1 3 3 0", + "test t a 0 1 2 1 4 4 0", + "test t idx 1 0 1 1 3 3 0", + "test t idx 1 1 2 1 4 4 0" + ] + ] + } +] diff --git a/pkg/table/tables/test/partition/partition_test.go b/pkg/table/tables/test/partition/partition_test.go new file mode 100644 index 0000000000000..4aea56803e93e --- /dev/null +++ b/pkg/table/tables/test/partition/partition_test.go @@ -0,0 +1,3367 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// 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 partition + +import ( + "context" + "fmt" + "math/rand" + "strconv" + "strings" + "testing" + gotime "time" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/sessiontxn" + "github.com/pingcap/tidb/pkg/table" + "github.com/pingcap/tidb/pkg/table/tables" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/testkit/testfailpoint" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util" + "github.com/pingcap/tidb/pkg/util/logutil" + "github.com/stretchr/testify/require" + "go.uber.org/zap" +) + +func TestPartitionAddRecord(t *testing.T) { + createTable1 := `CREATE TABLE test.t1 (id int(11), index(id)) +PARTITION BY RANGE ( id ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21) +)` + ctx := context.Background() + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + _, err := tk.Session().Execute(ctx, "use test") + require.NoError(t, err) + _, err = tk.Session().Execute(ctx, "drop table if exists t1, t2;") + require.NoError(t, err) + _, err = tk.Session().Execute(ctx, createTable1) + require.NoError(t, err) + tb, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + require.NoError(t, err) + tbInfo := tb.Meta() + p0 := tbInfo.Partition.Definitions[0] + require.Equal(t, model.NewCIStr("p0"), p0.Name) + require.Nil(t, sessiontxn.NewTxn(ctx, tk.Session())) + txn, err := tk.Session().Txn(true) + require.NoError(t, err) + rid, err := tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(1)) + require.NoError(t, err) + + // Check that add record writes to the partition, rather than the table. + val, err := txn.Get(context.TODO(), tables.PartitionRecordKey(p0.ID, rid.IntValue())) + require.NoError(t, err) + require.Greater(t, len(val), 0) + _, err = txn.Get(context.TODO(), tables.PartitionRecordKey(tbInfo.ID, rid.IntValue())) + require.True(t, kv.ErrNotExist.Equal(err)) + + // Cover more code. + _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(7)) + require.NoError(t, err) + _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(12)) + require.NoError(t, err) + _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(16)) + require.NoError(t, err) + + // Make the changes visible. + _, err = tk.Session().Execute(context.Background(), "commit") + require.NoError(t, err) + + // Check index count equals to data count. + tk.MustQuery("select count(*) from t1").Check(testkit.Rows("4")) + tk.MustQuery("select count(*) from t1 use index(id)").Check(testkit.Rows("4")) + tk.MustQuery("select count(*) from t1 use index(id) where id > 6").Check(testkit.Rows("3")) + + // Value must locates in one partition. + _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(22)) + require.True(t, table.ErrNoPartitionForGivenValue.Equal(err)) + _, err = tk.Session().Execute(context.Background(), "rollback") + require.NoError(t, err) + + createTable2 := `CREATE TABLE test.t2 (id int(11)) +PARTITION BY RANGE ( id ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p3 VALUES LESS THAN MAXVALUE +)` + _, err = tk.Session().Execute(context.Background(), createTable2) + require.NoError(t, err) + require.Nil(t, sessiontxn.NewTxn(ctx, tk.Session())) + txn, err = tk.Session().Txn(true) + require.NoError(t, err) + tb, err = dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + require.NoError(t, err) + _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(22)) + require.NoError(t, err) + + createTable3 := `create table test.t3 (id int) partition by range (id) + ( + partition p0 values less than (10) + )` + _, err = tk.Session().Execute(context.Background(), createTable3) + require.NoError(t, err) + require.Nil(t, sessiontxn.NewTxn(ctx, tk.Session())) + txn, err = tk.Session().Txn(true) + require.NoError(t, err) + tb, err = dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t3")) + require.NoError(t, err) + _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(11)) + require.True(t, table.ErrNoPartitionForGivenValue.Equal(err)) + _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(10)) + require.True(t, table.ErrNoPartitionForGivenValue.Equal(err)) + _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(0)) + require.NoError(t, err) + + createTable4 := `create table test.t4 (a int,b int) partition by range (a+b) + ( + partition p0 values less than (10) + );` + _, err = tk.Session().Execute(context.Background(), createTable4) + require.NoError(t, err) + require.Nil(t, sessiontxn.NewTxn(ctx, tk.Session())) + txn, err = tk.Session().Txn(true) + require.NoError(t, err) + tb, err = dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t4")) + require.NoError(t, err) + _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(1, 11)) + require.True(t, table.ErrNoPartitionForGivenValue.Equal(err)) +} + +func TestHashPartitionAddRecord(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + _, err := tk.Session().Execute(context.Background(), "use test") + require.NoError(t, err) + _, err = tk.Session().Execute(context.Background(), "drop table if exists t1;") + require.NoError(t, err) + _, err = tk.Session().Execute(context.Background(), "set @@session.tidb_enable_table_partition = '1';") + require.NoError(t, err) + _, err = tk.Session().Execute(context.Background(), `CREATE TABLE test.t1 (id int(11), index(id)) PARTITION BY HASH (id) partitions 4;`) + require.NoError(t, err) + tb, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + require.NoError(t, err) + tbInfo := tb.Meta() + p0 := tbInfo.Partition.Definitions[0] + require.Nil(t, sessiontxn.NewTxn(context.Background(), tk.Session())) + txn, err := tk.Session().Txn(true) + require.NoError(t, err) + rid, err := tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(8)) + require.NoError(t, err) + + // Check that add record writes to the partition, rather than the table. + val, err := txn.Get(context.TODO(), tables.PartitionRecordKey(p0.ID, rid.IntValue())) + require.NoError(t, err) + require.Greater(t, len(val), 0) + _, err = txn.Get(context.TODO(), tables.PartitionRecordKey(tbInfo.ID, rid.IntValue())) + require.True(t, kv.ErrNotExist.Equal(err)) + + // Cover more code. + _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(-1)) + require.NoError(t, err) + _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(3)) + require.NoError(t, err) + _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(6)) + require.NoError(t, err) + + // Make the changes visible. + _, err = tk.Session().Execute(context.Background(), "commit") + require.NoError(t, err) + + // Check index count equals to data count. + tk.MustQuery("select count(*) from t1").Check(testkit.Rows("4")) + tk.MustQuery("select count(*) from t1 use index(id)").Check(testkit.Rows("4")) + tk.MustQuery("select count(*) from t1 use index(id) where id > 2").Check(testkit.Rows("3")) + + // Test for partition expression is negative number. + _, err = tk.Session().Execute(context.Background(), `CREATE TABLE test.t2 (id int(11), index(id)) PARTITION BY HASH (id) partitions 11;`) + require.NoError(t, err) + tb, err = dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + require.NoError(t, err) + tbInfo = tb.Meta() + for i := 0; i < 11; i++ { + require.Nil(t, sessiontxn.NewTxn(context.Background(), tk.Session())) + txn, err = tk.Session().Txn(true) + require.NoError(t, err) + rid, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(-i)) + require.NoError(t, err) + val, err = txn.Get(context.TODO(), tables.PartitionRecordKey(tbInfo.Partition.Definitions[i].ID, rid.IntValue())) + require.NoError(t, err) + require.Greater(t, len(val), 0) + _, err = txn.Get(context.TODO(), tables.PartitionRecordKey(tbInfo.ID, rid.IntValue())) + require.True(t, kv.ErrNotExist.Equal(err)) + } + _, err = tk.Session().Execute(context.Background(), "drop table if exists t1, t2;") + require.NoError(t, err) +} + +// TestPartitionGetPhysicalID tests partition.GetPhysicalID(). +func TestPartitionGetPhysicalID(t *testing.T) { + createTable1 := `CREATE TABLE test.t1 (id int(11), index(id)) +PARTITION BY RANGE ( id ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21) +)` + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + _, err := tk.Session().Execute(context.Background(), "Drop table if exists test.t1;") + require.NoError(t, err) + _, err = tk.Session().Execute(context.Background(), createTable1) + require.NoError(t, err) + tb, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + require.NoError(t, err) + tbInfo := tb.Meta() + ps := tbInfo.GetPartitionInfo() + require.NotNil(t, ps) + for _, pd := range ps.Definitions { + p := tb.(table.PartitionedTable).GetPartition(pd.ID) + require.NotNil(t, p) + require.Equal(t, p.GetPhysicalID(), pd.ID) + } +} + +func TestGeneratePartitionExpr(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + _, err := tk.Session().Execute(context.Background(), "use test") + require.NoError(t, err) + _, err = tk.Session().Execute(context.Background(), "drop table if exists t1;") + require.NoError(t, err) + _, err = tk.Session().Execute(context.Background(), `create table t1 (id int) + partition by range (id) ( + partition p0 values less than (4), + partition p1 values less than (7), + partition p3 values less than maxvalue)`) + require.NoError(t, err) + + tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + require.NoError(t, err) + type partitionExpr interface { + PartitionExpr() *tables.PartitionExpr + } + pe := tbl.(partitionExpr).PartitionExpr() + + upperBounds := []string{ + "lt(t1.id, 4)", + "lt(t1.id, 7)", + "1", + } + for i, expr := range pe.UpperBounds { + require.Equal(t, upperBounds[i], expr.StringWithCtx(tk.Session().GetExprCtx().GetEvalCtx(), errors.RedactLogDisable)) + } +} + +func TestLocatePartition(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + + tk.MustExec(`CREATE TABLE t ( + id bigint(20) DEFAULT NULL, + type varchar(255) COLLATE utf8mb4_unicode_ci DEFAULT NULL + ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_unicode_ci + PARTITION BY LIST COLUMNS(type) + (PARTITION push_event VALUES IN ("PushEvent"), + PARTITION watch_event VALUES IN ("WatchEvent") + )`) + tk.MustExec(`insert into t values (1,"PushEvent"),(2,"WatchEvent"),(3, "WatchEvent")`) + tk.MustExec(`analyze table t all columns`) + + tk1 := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk3 := testkit.NewTestKit(t, store) + tks := []*testkit.TestKit{tk1, tk2, tk3} + + wg := util.WaitGroupWrapper{} + exec := func(tk0 *testkit.TestKit) { + tk0.MustExec("use test") + tk0.MustQuery("explain format = 'brief' select id, type from t where type = 'WatchEvent';").Check(testkit.Rows(""+ + `TableReader 2.00 root partition:watch_event data:Selection`, + `└─Selection 2.00 cop[tikv] eq(test.t.type, "WatchEvent")`, + ` └─TableFullScan 3.00 cop[tikv] table:t keep order:false`)) + } + + run := func(num int) { + tk := tks[num] + wg.Run(func() { + exec(tk) + }) + } + for i := 0; i < len(tks); i++ { + run(i) + } + wg.Wait() +} + +func TestIssue31629(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_enable_list_partition = 1") + tk.MustExec("create database Issue31629") + defer tk.MustExec("drop database Issue31629") + tk.MustExec("use Issue31629") + // Test following partition types: + // HASH, RANGE, LIST: + // - directly on a single int column + // - with expression on multiple columns + // RANGE/LIST COLUMNS single column + // RANGE/LIST COLUMNS -- Verify that only single column is allowed and no expression + tests := []struct { + create string + fail bool + cols []string + }{ + {"(col1 int, col2 varchar(60), col3 int, primary key(col1)) partition by range(col1) (partition p0 values less than (5),partition p1 values less than (10), partition p2 values less than maxvalue)", false, []string{"col1"}}, + {"(Col1 int, col2 varchar(60), col3 int, primary key(Col1,col3)) partition by range(Col1+col3) (partition p0 values less than (5),partition p1 values less than (10), partition p2 values less than maxvalue)", false, []string{"Col1", "col3"}}, + {"(col1 int, col2 varchar(60), col3 int, primary key(col1)) partition by hash(col1) partitions 3", false, []string{"col1"}}, + {"(Col1 int, col2 varchar(60), col3 int, primary key(Col1,col3)) partition by hash(Col1+col3) partitions 3", false, []string{"Col1", "col3"}}, + {"(col1 int, col2 varchar(60), col3 int, primary key(col1)) partition by list(col1) (partition p0 values in (5,6,7,8,9),partition p1 values in (10,11,12,13,14), partition p2 values in (20,21,22,23,24))", false, []string{"col1"}}, + {"(Col1 int, col2 varchar(60), col3 int, primary key(Col1,col3)) partition by list(Col1+col3) (partition p0 values in (5,6,7,8,9),partition p1 values in (10,11,12,13,14), partition p2 values in (20,21,22,23,24))", false, []string{"Col1", "col3"}}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by range columns (col2) (partition p0 values less than (""),partition p1 values less than ("MID"), partition p2 values less than maxvalue)`, false, []string{"col2"}}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by range columns (col2,col3) (partition p0 values less than (""),partition p1 values less than ("MID"), partition p2 values less than maxvalue)`, true, nil}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by range columns (col1+1) (partition p0 values less than (""),partition p1 values less than ("MID"), partition p2 values less than maxvalue)`, true, nil}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by list columns (col2) (partition p0 values in ("","First"),partition p1 values in ("MID","Middle"), partition p2 values in ("Last","Unknown"))`, false, []string{"col2"}}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by list columns (col2,col3) (partition p0 values in ("","First"),partition p1 values in ("MID","Middle"), partition p2 values in ("Last","Unknown"))`, true, nil}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by list columns (col1+1) (partition p0 values in ("","First"),partition p1 values in ("MID","Middle"), partition p2 values in ("Last","Unknown"))`, true, nil}, + } + + for i, tt := range tests { + createTable := "create table t1 " + tt.create + res, err := tk.Exec(createTable) + if res != nil { + res.Close() + } + if err != nil { + if tt.fail { + continue + } + } + require.Falsef(t, tt.fail, "test %d succeeded but was expected to fail! %s", i, createTable) + require.NoError(t, err) + tk.MustQuery("show warnings").Check(testkit.Rows()) + + tb, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("Issue31629"), model.NewCIStr("t1")) + require.NoError(t, err) + tbp, ok := tb.(table.PartitionedTable) + require.Truef(t, ok, "test %d does not generate a table.PartitionedTable: %s (%T, %+v)", i, createTable, tb, tb) + colNames := tbp.GetPartitionColumnNames() + checkNames := []model.CIStr{model.NewCIStr(tt.cols[0])} + for i := 1; i < len(tt.cols); i++ { + checkNames = append(checkNames, model.NewCIStr(tt.cols[i])) + } + require.ElementsMatchf(t, colNames, checkNames, "test %d %s", i, createTable) + tk.MustExec("drop table t1") + } +} + +func TestExchangePartitionStates(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + dbName := "partSchemaVer" + tk.MustExec("create database " + dbName) + tk.MustExec("use " + dbName) + tk.MustExec(`set @@global.tidb_enable_metadata_lock = ON`) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use " + dbName) + tk3 := testkit.NewTestKit(t, store) + tk3.MustExec("use " + dbName) + tk4 := testkit.NewTestKit(t, store) + tk4.MustExec("use " + dbName) + tk.MustExec(`create table t (a int primary key, b varchar(255), key (b))`) + tk.MustExec(`create table tp (a int primary key, b varchar(255), key (b)) partition by range (a) (partition p0 values less than (1000000), partition p1M values less than (2000000))`) + tk.MustExec(`insert into t values (1, "1")`) + tk.MustExec(`insert into tp values (2, "2")`) + tk.MustExec(`analyze table t,tp`) + tk.MustExec("BEGIN") + tk.MustQuery(`select * from t`).Check(testkit.Rows("1 1")) + tk.MustQuery(`select * from tp`).Check(testkit.Rows("2 2")) + alterChan := make(chan error) + go func() { + // WITH VALIDATION is the default + err := tk2.ExecToErr(`alter table tp exchange partition p0 with table t`) + alterChan <- err + }() + waitFor := func(tableName, s string, pos int) { + for { + select { + case alterErr := <-alterChan: + require.Fail(t, "Alter completed unexpectedly", "With error %v", alterErr) + default: + // Alter still running + } + res := tk4.MustQuery(`admin show ddl jobs where db_name = '` + strings.ToLower(dbName) + `' and table_name = '` + tableName + `' and job_type = 'exchange partition'`).Rows() + if len(res) == 1 && res[0][pos] == s { + logutil.BgLogger().Info("Got state", zap.String("State", s)) + break + } + gotime.Sleep(10 * gotime.Millisecond) + } + dom := domain.GetDomain(tk.Session()) + // Make sure the table schema is the new schema. + require.NoError(t, dom.Reload()) + } + waitFor("t", "write only", 4) + tk3.MustExec(`BEGIN`) + tk3.MustExec(`insert into t values (4,"4")`) + tk3.MustContainErrMsg(`insert into t values (1000004,"1000004")`, "[table:1748]Found a row not matching the given partition set") + tk.MustExec(`insert into t values (5,"5")`) + // This should fail the alter table! + tk.MustExec(`insert into t values (1000005,"1000005")`) + + // MDL will block the alter to not continue until all clients + // are in StateWriteOnly, which tk is blocking until it commits + tk.MustExec(`COMMIT`) + waitFor("t", "rollback done", 11) + // MDL will block the alter from finish, tk is in 'rollbacked' schema version + // but the alter is still waiting for tk3 to commit, before continuing + tk.MustExec("BEGIN") + tk.MustExec(`insert into t values (1000006,"1000006")`) + tk.MustExec(`insert into t values (6,"6")`) + tk3.MustExec(`insert into t values (7,"7")`) + tk3.MustContainErrMsg(`insert into t values (1000007,"1000007")`, + "[table:1748]Found a row not matching the given partition set") + tk3.MustExec("COMMIT") + require.ErrorContains(t, <-alterChan, + "[ddl:1737]Found a row that does not match the partition") + tk3.MustExec(`BEGIN`) + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows( + "1 1", "1000005 1000005", "1000006 1000006", "5 5", "6 6")) + tk.MustQuery(`select * from tp`).Sort().Check(testkit.Rows("2 2")) + tk3.MustQuery(`select * from t`).Sort().Check(testkit.Rows( + "1 1", "1000005 1000005", "4 4", "5 5", "7 7")) + tk3.MustQuery(`select * from tp`).Sort().Check(testkit.Rows("2 2")) + tk.MustContainErrMsg(`insert into t values (7,"7")`, + "[kv:1062]Duplicate entry '7' for key 't.PRIMARY'") + tk.MustExec(`insert into t values (8,"8")`) + tk.MustExec(`insert into t values (1000008,"1000008")`) + tk.MustExec(`insert into tp values (9,"9")`) + tk.MustExec(`insert into tp values (1000009,"1000009")`) + tk3.MustExec(`insert into t values (10,"10")`) + tk3.MustExec(`insert into t values (1000010,"1000010")`) + + tk3.MustExec(`COMMIT`) + tk.MustQuery(`show create table tp`).Check(testkit.Rows("" + + "tp CREATE TABLE `tp` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (1000000),\n" + + " PARTITION `p1M` VALUES LESS THAN (2000000))")) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tk.MustExec(`commit`) + tk.MustExec(`insert into t values (11,"11")`) + tk.MustExec(`insert into t values (1000011,"1000011")`) + tk.MustExec(`insert into tp values (12,"12")`) + tk.MustExec(`insert into tp values (1000012,"1000012")`) +} + +// Test partition and non-partition both have check constraints. +func TestExchangePartitionCheckConstraintStates(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec(`create database check_constraint`) + tk.MustExec(`set @@global.tidb_enable_check_constraint = 1`) + tk.MustExec(`use check_constraint`) + tk.MustExec(`create table nt (a int check (a > 75) not ENFORCED, b int check (b > 50) ENFORCED)`) + tk.MustExec(`create table pt (a int check (a < 75) ENFORCED, b int check (b < 75) ENFORCED) partition by range (a) (partition p0 values less than (50), partition p1 values less than (100) )`) + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec(`use check_constraint`) + tk3 := testkit.NewTestKit(t, store) + tk3.MustExec(`use check_constraint`) + tk4 := testkit.NewTestKit(t, store) + tk4.MustExec(`use check_constraint`) + // TODO: error message to check. + errMsg := "[table:3819]Check constraint" + + tk2.MustExec("begin") + // Get table mdl. + tk2.MustQuery(`select * from nt`).Check(testkit.Rows()) + tk2.MustQuery(`select * from pt`).Check(testkit.Rows()) + alterChan := make(chan error) + go func() { + err := tk3.ExecToErr(`alter table pt exchange partition p1 with table nt`) + alterChan <- err + }() + waitFor := func(tableName, s string, pos int) { + for { + select { + case alterErr := <-alterChan: + require.Fail(t, "Alter completed unexpectedly", "With error %v", alterErr) + default: + // Alter still running + } + res := tk4.MustQuery(`admin show ddl jobs where db_name = 'check_constraint' and table_name = '` + tableName + `' and job_type = 'exchange partition'`).Rows() + if len(res) == 1 && res[0][pos] == s { + logutil.BgLogger().Info("Got state", zap.String("State", s)) + break + } + gotime.Sleep(10 * gotime.Millisecond) + } + dom := domain.GetDomain(tk.Session()) + // Make sure the table schema is the new schema. + require.NoError(t, dom.Reload()) + } + waitFor("nt", "write only", 4) + + tk.MustExec(`insert into nt values (60, 60)`) + // violate pt (a < 75) + tk.MustContainErrMsg(`insert into nt values (80, 60)`, errMsg) + // violate pt (b < 75) + tk.MustContainErrMsg(`insert into nt values (60, 80)`, errMsg) + // violate pt (a < 75) + tk.MustContainErrMsg(`update nt set a = 80 where a = 60`, errMsg) + // violate pt (b < 75) + tk.MustContainErrMsg(`update nt set b = 80 where b = 60`, errMsg) + + tk.MustExec(`insert into pt values (60, 60)`) + // violate nt (b > 50) + tk.MustContainErrMsg(`insert into pt values (60, 50)`, errMsg) + // violate nt (b > 50) + tk.MustContainErrMsg(`update pt set b = 50 where b = 60`, errMsg) + // row in partition p0(less than (50)), is ok. + tk.MustExec(`insert into pt values (30, 50)`) + + tk5 := testkit.NewTestKit(t, store) + tk5.MustExec(`use check_constraint`) + tk5.MustExec("begin") + // Let tk5 get mdl of pt with the version of write-only state. + tk5.MustQuery(`select * from pt`) + + tk6 := testkit.NewTestKit(t, store) + tk6.MustExec(`use check_constraint`) + tk6.MustExec("begin") + // Let tk6 get mdl of nt with the version of write-only state. + tk6.MustQuery(`select * from nt`) + + // Release tk2 mdl, wait ddl enter next state. + tk2.MustExec("commit") + waitFor("pt", "none", 4) + + // violate nt (b > 50) + // Now tk5 handle the sql with MDL: pt version state is write-only, nt version state is none. + tk5.MustContainErrMsg(`insert into pt values (60, 50)`, errMsg) + // Verify exists row(60, 60) in pt. + tk5.MustQuery(`select * from pt where a = 60 and b = 60`).Check(testkit.Rows("60 60")) + // Update oldData and newData both in p1, violate nt (b > 50) + tk5.MustContainErrMsg(`update pt set b = 50 where a = 60 and b = 60`, errMsg) + // Verify exists row(30, 50) in pt. + tk5.MustQuery(`select * from pt where a = 30 and b = 50`).Check(testkit.Rows("30 50")) + // update oldData in p0, newData in p1, violate nt (b > 50) + tk5.MustContainErrMsg(`update pt set a = 60 where a = 30 and b = 50`, errMsg) + + // violate pt (a < 75) + tk6.MustContainErrMsg(`insert into nt values (80, 60)`, errMsg) + // violate pt (b < 75) + tk6.MustContainErrMsg(`insert into nt values (60, 80)`, errMsg) + // Verify exists row(60, 60) in nt. + tk6.MustQuery(`select * from pt where a = 60 and b = 60`).Check(testkit.Rows("60 60")) + // violate pt (a < 75) + tk6.MustContainErrMsg(`update nt set a = 80 where a = 60 and b = 60`, errMsg) + + // Let tk5, tk6 release mdl. + tk5.MustExec("commit") + tk6.MustExec("commit") + + // Wait ddl finish. + <-alterChan +} + +// Test partition table has check constraints while non-partition table do not have. +func TestExchangePartitionCheckConstraintStatesTwo(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec(`create database check_constraint`) + tk.MustExec(`set @@global.tidb_enable_check_constraint = 1`) + tk.MustExec(`use check_constraint`) + tk.MustExec(`create table nt (a int, b int)`) + tk.MustExec(`create table pt (a int check (a < 75) ENFORCED, b int check (b < 75) ENFORCED) partition by range (a) (partition p0 values less than (50), partition p1 values less than (100) )`) + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec(`use check_constraint`) + tk3 := testkit.NewTestKit(t, store) + tk3.MustExec(`use check_constraint`) + tk4 := testkit.NewTestKit(t, store) + tk4.MustExec(`use check_constraint`) + // TODO: error message to check. + errMsg := "[table:3819]Check constraint" + + tk2.MustExec("begin") + // Get table mdl. + tk2.MustQuery(`select * from nt`).Check(testkit.Rows()) + alterChan := make(chan error) + go func() { + err := tk3.ExecToErr(`alter table pt exchange partition p1 with table nt`) + alterChan <- err + }() + waitFor := func(tableName, s string, pos int) { + for { + select { + case alterErr := <-alterChan: + require.Fail(t, "Alter completed unexpectedly", "With error %v", alterErr) + default: + // Alter still running + } + res := tk4.MustQuery(`admin show ddl jobs where db_name = 'check_constraint' and table_name = '` + tableName + `' and job_type = 'exchange partition'`).Rows() + if len(res) == 1 && res[0][pos] == s { + logutil.BgLogger().Info("Got state", zap.String("State", s)) + break + } + gotime.Sleep(10 * gotime.Millisecond) + } + dom := domain.GetDomain(tk.Session()) + // Make sure the table schema is the new schema. + require.NoError(t, dom.Reload()) + } + waitFor("nt", "write only", 4) + + tk.MustExec(`insert into nt values (60, 60)`) + // violate pt (a < 75) + tk.MustContainErrMsg(`insert into nt values (80, 60)`, errMsg) + // violate pt (b < 75) + tk.MustContainErrMsg(`insert into nt values (60, 80)`, errMsg) + // violate pt (a < 75) + tk.MustContainErrMsg(`update nt set a = 80 where a = 60`, errMsg) + // violate pt (b < 75) + tk.MustContainErrMsg(`update nt set b = 80 where b = 60`, errMsg) + + tk.MustExec(`insert into pt values (60, 60)`) + tk.MustExec(`insert into pt values (60, 50)`) + tk.MustExec(`update pt set b = 50 where b = 60`) + // row in partition p0(less than (50)), is ok. + tk.MustExec(`insert into pt values (30, 50)`) + + // Release tk2 mdl. + tk2.MustExec("commit") + // Wait ddl finish. + <-alterChan +} + +func TestAddKeyPartitionStates(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + dbName := "partSchemaVer" + tk.MustExec("create database " + dbName) + tk.MustExec("use " + dbName) + tk.MustExec(`set @@global.tidb_enable_metadata_lock = ON`) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use " + dbName) + tk3 := testkit.NewTestKit(t, store) + tk3.MustExec("use " + dbName) + tk4 := testkit.NewTestKit(t, store) + tk4.MustExec("use " + dbName) + tk.MustExec(`create table t (a int primary key, b varchar(255), key (b)) partition by hash (a) partitions 3`) + tk.MustExec(`insert into t values (1, "1")`) + tk.MustExec(`analyze table t`) + tk.MustExec("BEGIN") + tk.MustQuery(`select * from t`).Check(testkit.Rows("1 1")) + tk.MustExec(`insert into t values (2, "2")`) + syncChan := make(chan bool) + go func() { + tk2.MustExec(`alter table t add partition partitions 1`) + syncChan <- true + }() + waitFor := func(i int, s string) { + for { + res := tk4.MustQuery(`admin show ddl jobs where db_name = '` + strings.ToLower(dbName) + `' and table_name = 't' and job_type like 'alter table%'`).Rows() + if len(res) == 1 && res[0][i] == s { + break + } + gotime.Sleep(10 * gotime.Millisecond) + } + dom := domain.GetDomain(tk.Session()) + // Make sure the table schema is the new schema. + require.NoError(t, dom.Reload()) + } + waitFor(4, "delete only") + tk3.MustExec(`BEGIN`) + tk3.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1")) + tk3.MustExec(`insert into t values (3,"3")`) + + tk.MustExec(`COMMIT`) + waitFor(4, "write only") + tk.MustExec(`BEGIN`) + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "2 2")) + tk.MustExec(`insert into t values (4,"4")`) + + tk3.MustExec(`COMMIT`) + waitFor(4, "write reorganization") + tk3.MustExec(`BEGIN`) + tk3.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY HASH (`a`) PARTITIONS 3")) + tk3.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "2 2", "3 3")) + tk3.MustExec(`insert into t values (5,"5")`) + + tk.MustExec(`COMMIT`) + waitFor(4, "delete reorganization") + tk.MustExec(`BEGIN`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY HASH (`a`) PARTITIONS 4")) + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "2 2", "3 3", "4 4")) + tk.MustExec(`insert into t values (6,"6")`) + + tk3.MustExec(`COMMIT`) + tk.MustExec(`COMMIT`) + <-syncChan + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 1", "2 2", "3 3", "4 4", "5 5", "6 6")) +} + +type compoundSQL struct { + selectSQL string + point bool + batchPoint bool + pruned bool + executeExplain bool + usedPartition []string + notUsedPartition []string + rowCount int +} + +type partTableCase struct { + partitionbySQL string + selectInfo []compoundSQL +} + +func executePartTableCase(t *testing.T, tk *testkit.TestKit, testCases []partTableCase, + createSQL string, insertSQLs []string, dropSQL string) { + for i, testCase := range testCases { + // create table ... partition by key ... + ddlSQL := createSQL + testCase.partitionbySQL + logutil.BgLogger().Info("Partition DDL test", zap.Int("i", i), zap.String("ddlSQL", ddlSQL)) + executeSQLWrapper(t, tk, ddlSQL) + // insert data + for _, insertsql := range insertSQLs { + executeSQLWrapper(t, tk, insertsql) + } + // execute testcases + for j, selInfo := range testCase.selectInfo { + logutil.BgLogger().Info("Select", zap.Int("j", j), zap.String("selectSQL", selInfo.selectSQL)) + tk.MustQuery(selInfo.selectSQL).Check(testkit.Rows(strconv.Itoa(selInfo.rowCount))) + if selInfo.executeExplain { + result := tk.MustQuery("EXPLAIN " + selInfo.selectSQL) + if selInfo.point { + result.CheckContain("Point_Get") + } + if selInfo.batchPoint { + result.CheckContain("Batch_Point_Get") + } + if selInfo.pruned { + for _, part := range selInfo.usedPartition { + result.CheckContain(part) + } + for _, part := range selInfo.notUsedPartition { + result.CheckNotContain(part) + } + } + } + } + executeSQLWrapper(t, tk, dropSQL) + } +} + +func executeSQLWrapper(t *testing.T, tk *testkit.TestKit, SQLString string) { + res, err := tk.Exec(SQLString) + if res != nil { + res.Close() + } + require.Nil(t, err) +} + +func TestKeyPartitionTableBasic(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database partitiondb") + defer tk.MustExec("drop database partitiondb") + tk.MustExec("use partitiondb") + testCases := []struct { + createSQL string + dropSQL string + insertSQL string + selectInfo []compoundSQL + }{ + { + createSQL: "CREATE TABLE tkey0 (col1 INT NOT NULL, col2 DATE NOT NULL, col3 INT NOT NULL, col4 INT NOT NULL,UNIQUE KEY (col3)) PARTITION BY KEY(col3) PARTITIONS 4", + insertSQL: "INSERT INTO tkey0 VALUES(1, '2023-02-22', 1, 1), (2, '2023-02-22', 2, 2), (3, '2023-02-22', 3, 3), (4, '2023-02-22', 4, 4)", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey0", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey0 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey0 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey0 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 0, + }, + { + "SELECT count(*) FROM tkey0 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey0 WHERE col3 = 3", + true, false, true, true, []string{"partition:p3"}, []string{"partition:p0,p1,p2"}, 1, + }, + { + "SELECT count(*) FROM tkey0 WHERE col3 = 3 or col3 = 4", + false, false, true, true, []string{"partition:p0,p3"}, []string{"partition:p1,p2"}, 2, + }, + { + "SELECT count(*) FROM tkey0 WHERE col3 >1 AND col3 < 4", + false, false, true, true, []string{"partition:p1,p3"}, []string{"partition:p0,p2"}, 2, + }, + }, + + dropSQL: "DROP TABLE IF EXISTS tkey0", + }, + { + createSQL: "CREATE TABLE tkey7 (col1 INT NOT NULL, col2 DATE NOT NULL, col3 INT NOT NULL, col4 INT NOT NULL,UNIQUE KEY (col3,col1)) PARTITION BY KEY(col3,col1) PARTITIONS 4", + insertSQL: "INSERT INTO tkey7 VALUES(1, '2023-02-22', 1, 1), (1, '2023-02-22', 2, 1),(2, '2023-02-22', 2, 2), (3, '2023-02-22', 3, 3), (4, '2023-02-22', 4, 4),(4, '2023-02-22', 5, 4)", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey7", + false, false, false, false, []string{}, []string{}, 6, + }, + { + "SELECT count(*) FROM tkey7 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey7 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey7 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey7 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey7 WHERE col3 = 3", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey7 WHERE col3 = 3 and col1 = 3", + true, false, true, true, []string{"partition:p1"}, []string{"partition:p1,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey7 WHERE col3 = 3 or col3 = 4", + false, false, true, true, []string{"partition:all"}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey7 WHERE col3 = 3 and col1 = 3 OR col3 = 4 and col1 = 4", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p1,p2,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey7 WHERE col1>1 and col3 >1 AND col3 < 4 and col1<3", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey7", + }, + { + createSQL: "CREATE TABLE tkey8 (col1 INT NOT NULL, col2 DATE NOT NULL, col3 INT NOT NULL, col4 INT NOT NULL,PRIMARY KEY (col3,col1)) PARTITION BY KEY(col3,col1) PARTITIONS 4", + insertSQL: "INSERT INTO tkey8 VALUES(1, '2023-02-22', 111, 1), (1, '2023-02-22', 2, 1),(2, '2023-02-22', 218, 2), (3, '2023-02-22', 3, 3), (4, '2023-02-22', 4, 4),(4, '2023-02-22', 5, 4),(5, '2023-02-22', 5, 5),(5, '2023-02-22', 50, 2),(6, '2023-02-22', 62, 2),(60, '2023-02-22', 6, 5),(70, '2023-02-22', 50, 2),(80, '2023-02-22', 62, 2),(100, '2023-02-22', 62, 2),(2000, '2023-02-22', 6, 5),(400, '2023-02-22', 50, 2),(90, '2023-02-22', 62, 2)", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey8", + false, false, false, false, []string{}, []string{}, 16, + }, + { + "SELECT count(*) FROM tkey8 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey8 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 7, + }, + { + "SELECT count(*) FROM tkey8 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey8 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey8 WHERE col3 = 3", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey8 WHERE col3 = 3 and col1 = 3", + true, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey8 WHERE col3 = 3 or col3 = 4", + false, false, true, true, []string{"partition:all"}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey8 WHERE col3 = 3 and col1 = 3 OR col3 = 4 and col1 = 4", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey8 WHERE col1>1 and col3 >1 AND col3 < 4 and col1<3", + false, false, true, true, []string{"partition:all"}, []string{}, 0, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey8", + }, + { + createSQL: "CREATE TABLE tkey6 (col1 INT NOT NULL, col2 DATE NOT NULL, col3 VARCHAR(12) NOT NULL, col4 INT NOT NULL,UNIQUE KEY (col3)) PARTITION BY KEY(col3) PARTITIONS 4", + insertSQL: "INSERT INTO tkey6 VALUES(1, '2023-02-22', 'linpin', 1), (2, '2023-02-22', 'zhangsan', 2), (3, '2023-02-22', 'anqila', 3), (4, '2023-02-22', 'xingtian', 4),(1, '2023-02-22', 'renleifeng', 5), (2, '2023-02-22', 'peilin', 2),(1, '2023-02-22', 'abcdeeg', 7), (2, '2023-02-22', 'rpstdfed', 8)", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey6", + false, false, false, false, []string{}, []string{}, 8, + }, + { + "SELECT count(*) FROM tkey6 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey6 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey6 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey6 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey6 WHERE col3 = 'linpin'", + true, false, true, true, []string{"partition:p3"}, []string{"partition:p0,p1,p2"}, 1, + }, + { + "SELECT count(*) FROM tkey6 WHERE col3 = 'zhangsan' or col3 = 'linpin'", + false, false, true, true, []string{"partition:p2,p3"}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey6 WHERE col3 > 'linpin' AND col3 < 'qing'", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey6", + }, + { + createSQL: "CREATE TABLE tkey2 (JYRQ INT not null,KHH VARCHAR(12) not null,ZJZH CHAR(14) not null,primary key (JYRQ, KHH, ZJZH))PARTITION BY KEY(KHH) partitions 4", + insertSQL: "INSERT INTO tkey2 VALUES(1,'nanjing','025'),(2,'huaian','0517'),(3,'zhenjiang','0518'),(4,'changzhou','0519'),(5,'wuxi','0511'),(6,'suzhou','0512'),(7,'xuzhou','0513'),(8,'suqian','0513'),(9,'lianyungang','0514'),(10,'yangzhou','0515'),(11,'taizhou','0516'),(12,'nantong','0520'),(13,'yancheng','0521'),(14,'NANJING','025'),(15,'HUAIAN','0527'),(16,'ZHENJIANG','0529'),(17,'CHANGZHOU','0530')", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey2", + false, false, false, false, []string{}, []string{}, 17, + }, + { + "SELECT count(*) FROM tkey2 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey2 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey2 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey2 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 6, + }, + { + "SELECT count(*) FROM tkey2 WHERE KHH = 'huaian'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p0,p1,p2"}, 1, + }, + { + "SELECT count(*) FROM tkey2 WHERE KHH = 'huaian' or KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p0,p1,p2"}, 2, + }, + { + "SELECT count(*) FROM tkey2 WHERE KHH > 'nanjing' AND KHH < 'suzhou'", + false, false, true, true, []string{"partition:all"}, []string{}, 2, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey2", + }, + { + createSQL: "CREATE TABLE tkey5 (JYRQ INT not null,KHH VARCHAR(12) not null,ZJZH CHAR(14) not null,primary key (KHH, JYRQ, ZJZH))PARTITION BY KEY(KHH) partitions 4", + insertSQL: "INSERT INTO tkey5 VALUES(1,'nanjing','025'),(2,'huaian','0517'),(3,'zhenjiang','0518'),(4,'changzhou','0519'),(5,'wuxi','0511'),(6,'suzhou','0512'),(7,'xuzhou','0513'),(8,'suqian','0513'),(9,'lianyungang','0514'),(10,'yangzhou','0515'),(11,'taizhou','0516'),(12,'nantong','0520'),(13,'yancheng','0521'),(14,'NANJING','025'),(15,'HUAIAN','0527'),(16,'ZHENJIANG','0529'),(17,'CHANGZHOU','0530')", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey5", + false, false, false, false, []string{}, []string{}, 17, + }, + { + "SELECT count(*) FROM tkey5 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey5 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey5 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey5 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 6, + }, + { + "SELECT count(*) FROM tkey5 WHERE KHH = 'huaian'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p0,p1,p2"}, 1, + }, + { + "SELECT count(*) FROM tkey5 WHERE KHH = 'huaian' or KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p0,p1,p2"}, 2, + }, + { + "SELECT count(*) FROM tkey5 WHERE KHH > 'nanjing' AND KHH < 'suzhou'", + false, false, true, true, []string{"partition:all"}, []string{}, 2, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey5", + }, + { + createSQL: "CREATE TABLE tkey4 (JYRQ INT not null,KHH VARCHAR(12) not null,ZJZH CHAR(14) not null,primary key (JYRQ, KHH, ZJZH))PARTITION BY KEY(JYRQ, KHH) partitions 4", + insertSQL: "INSERT INTO tkey4 VALUES(1,'nanjing','025'),(2,'huaian','0517'),(3,'zhenjiang','0518'),(4,'changzhou','0519'),(5,'wuxi','0511'),(6,'suzhou','0512'),(7,'xuzhou','0513'),(8,'suqian','0513'),(9,'lianyungang','0514'),(10,'yangzhou','0515'),(11,'taizhou','0516'),(12,'nantong','0520'),(13,'yancheng','0521'),(14,'NANJING','025'),(15,'HUAIAN','0527'),(16,'ZHENJIANG','0529'),(17,'CHANGZHOU','0530'),(1,'beijing','010'),(2,'beijing','010'),(2,'zzzzwuhan','027')", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey4", + false, false, false, false, []string{}, []string{}, 20, + }, + { + "SELECT count(*) FROM tkey4 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 7, + }, + { + "SELECT count(*) FROM tkey4 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey4 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey4 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian'", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ = 2", + false, false, true, true, []string{"partition:all"}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian' and JYRQ = 2", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian' and JYRQ = 2 or KHH = 'zhenjiang' and JYRQ = 3", + false, false, true, true, []string{"partition:p0,p1"}, []string{"partition:p2,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian' and JYRQ = 2 or KHH = 'zhenjiang' and JYRQ = 3 or KHH = 'HUAIAN' and JYRQ = 15", + false, false, true, true, []string{"partition:p0,p1"}, []string{"partition:p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian' or KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:all"}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ = 2 OR JYRQ = 3", + false, false, true, true, []string{"partition:all"}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ = 2 OR JYRQ = 3 OR JYRQ = 15", + false, false, true, true, []string{"partition:all"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ >6 AND JYRQ < 10", + false, false, true, true, []string{"partition:all"}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ >6 and KHH>'lianyungang' AND JYRQ < 10 and KHH<'xuzhou'", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey4", + }, + { + createSQL: "CREATE TABLE tkey9 (JYRQ INT not null,KHH VARCHAR(12) not null,ZJZH CHAR(14) not null,primary key (JYRQ, KHH, ZJZH))PARTITION BY KEY(JYRQ, KHH, ZJZH) partitions 4", + insertSQL: "INSERT INTO tkey9 VALUES(1,'nanjing','025'),(2,'huaian','0517'),(3,'zhenjiang','0518'),(4,'changzhou','0519'),(5,'wuxi','0511'),(6,'suzhou','0512'),(7,'xuzhou','0513'),(8,'suqian','0513'),(9,'lianyungang','0514'),(10,'yangzhou','0515'),(11,'taizhou','0516'),(12,'nantong','0520'),(13,'yancheng','0521'),(14,'NANJING','025'),(15,'HUAIAN','0527'),(16,'ZHENJIANG','0529'),(17,'CHANGZHOU','0530'),(1,'beijing','010'),(2,'beijing','010'),(2,'zzzzwuhan','027')", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey9", + false, false, false, false, []string{}, []string{}, 20, + }, + { + "SELECT count(*) FROM tkey9 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 6, + }, + { + "SELECT count(*) FROM tkey9 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey9 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey9 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 8, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' and JYRQ = 2 and ZJZH = '0517'", + true, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' and JYRQ = 2", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ = 2", + false, false, true, true, []string{"partition:all"}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' and JYRQ = 2 and ZJZH='0517' or KHH = 'zhenjiang' and JYRQ = 3 and ZJZH = '0518'", + false, false, true, true, []string{"partition:p0,p3"}, []string{"partition:p1,p2"}, 2, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' and JYRQ = 2 and ZJZH='0517' or KHH = 'zhenjiang' and JYRQ = 3 and ZJZH = '0518' or KHH = 'NANJING' and JYRQ = 14 and ZJZH = '025'", + false, false, true, true, []string{"partition:p0,p3"}, []string{"partition:p2,p1"}, 3, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' or KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:all"}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ = 2 OR JYRQ = 3", + false, false, true, true, []string{"partition:all"}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ = 2 OR JYRQ = 3 OR JYRQ = 15", + false, false, true, true, []string{"partition:all"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ >6 AND JYRQ < 10", + false, false, true, true, []string{"partition:all"}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ = 2 and KHH = 'huaian' OR JYRQ = 3 and KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:all"}, []string{}, 2, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey9", + }, + } + + for i, testCase := range testCases { + logutil.BgLogger().Info("Partition DDL test", zap.Int("i", i), zap.String("createSQL", testCase.createSQL)) + executeSQLWrapper(t, tk, testCase.createSQL) + executeSQLWrapper(t, tk, testCase.insertSQL) + for j, selInfo := range testCase.selectInfo { + logutil.BgLogger().Info("Select", zap.Int("j", j), zap.String("selectSQL", selInfo.selectSQL)) + tk.MustQuery(selInfo.selectSQL).Check(testkit.Rows(strconv.Itoa(selInfo.rowCount))) + if selInfo.executeExplain { + result := tk.MustQuery("EXPLAIN " + selInfo.selectSQL) + if selInfo.point { + result.CheckContain("Point_Get") + } + if selInfo.batchPoint { + result.CheckContain("Batch_Point_Get") + } + if selInfo.pruned { + for _, part := range selInfo.usedPartition { + result.CheckContain(part) + } + } + } + } + executeSQLWrapper(t, tk, testCase.dropSQL) + } +} + +func TestKeyPartitionTableAllFeildType(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database partitiondb3") + defer tk.MustExec("drop database partitiondb3") + tk.MustExec("use partitiondb3") + // partition column is numeric family + createSQL := "create table tkey_numeric(\n" + + "id1 BIT(8) not null,\n" + + "id2 TINYINT not null,\n" + + "id3 BOOL not null,\n" + + "id4 SMALLINT not null,\n" + + "id5 MEDIUMINT not null,\n" + + "id6 INT not null,\n" + + "id7 BIGINT not null,\n" + + "id8 DECIMAL(12,4) not null,\n" + + "id9 FLOAT not null,\n" + + "id10 DOUBLE not null,\n" + + "name varchar(20),\n" + + "primary key(id1,id2,id3,id4,id5,id6,id7,id8,id9,id10)\n" + + ")\n" + dropSQL := "drop table tkey_numeric" + insertSQLS := []string{ + "INSERT INTO tkey_numeric VALUES(1,1,0,1,1,1,1,1.1,120.1,367.45,'linpin'),(12,12,12,12,12,12,12,12.1,1220.1,3267.45,'anqila')", + "INSERT INTO tkey_numeric VALUES(0,2,1,2,2,2,2,2.78,16.78,17.25,'ring'),(33,33,33,33,33,33,33,33.78,336.78,37.25,'black')", + "INSERT INTO tkey_numeric VALUES(2,3,1,3,3,3,3,3.78,26.78,417.25,'liudehua'),(22,23,21,23,23,23,23,32.78,26.72,27.15,'chenchen')", + "INSERT INTO tkey_numeric VALUES(3,3,2,4,4,4,4,4.78,46.48,89.35,'guofucheng'), (4,4,4,5,5,5,5,5.78,56.48,59.35,'zhangxuyou')", + "INSERT INTO tkey_numeric VALUES(5,5,5,5,5,5,5,5.78,56.48,59.35,'xietingfeng'),(34,34,34,34,34,34,34,34.78,346.78,34.25,'dongxu')", + "INSERT INTO tkey_numeric VALUES(250,120,120,250,250,258,348,38.78,186.48,719.35,'chenguanxi'),(35,35,35,250,35,35,35,35.78,356.48,35.35,'chenguanxi')", + } + testCases := []partTableCase{ + { + partitionbySQL: "PARTITION BY KEY(id1) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:all"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 5, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id1 = 3", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 1, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id1 = 3 or id1 = 4", + false, false, true, true, []string{"partition:p0,p3"}, []string{"partition:p1,p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id1 >1 AND id1 < 4", + false, false, true, true, []string{"partition:p1,p3"}, []string{"partition:p2,p0"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id2) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:all"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id2 = 3", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p0,p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id2 = 3 or id2 = 4", + false, false, true, true, []string{"partition:p0,p3"}, []string{"partition:p1,p2"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id2 >1 AND id2 < 4", + false, false, true, true, []string{"partition:p1,p3"}, []string{"partition:p0,p2"}, 3, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id3) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:all"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p0,p1,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id3 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id3 = 5 or id3 = 4", + false, false, true, true, []string{"partition:p0,p2"}, []string{"partition:p1,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id3 >1 AND id3 < 4", + false, false, true, true, []string{"partition:p1,p3"}, []string{"partition:p2,p0"}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id4) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:all"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 5, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id4 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id4 = 5 or id4 = 4", + false, false, true, true, []string{"partition:p0,p2"}, []string{"partition:p1,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id4 >1 AND id4 < 4", + false, false, true, true, []string{"partition:p1,p3"}, []string{"partition:p0,p2"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id5) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:all"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p3,p0"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id5 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id5 = 5 or id5 = 4", + false, false, true, true, []string{"partition:p0,p2"}, []string{"partition:p1,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id5 >1 AND id5 < 4", + false, false, true, true, []string{"partition:p1,p3"}, []string{"partition:p2,p0"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id6) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:all"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id6 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id6 = 5 or id6 = 4", + false, false, true, true, []string{"partition:p0,p2"}, []string{"partition:p1,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id6 >1 AND id6 < 4", + false, false, true, true, []string{"partition:p1,p3"}, []string{"partition:p2,p0"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id7) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:all"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id7 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id7 = 5 or id7 = 4", + false, false, true, true, []string{"partition:p0,p2"}, []string{"partition:p1,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id7 >1 AND id7 < 4", + false, false, true, true, []string{"partition:p1,p3"}, []string{"partition:p2,p0"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id8) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:all"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id8 = 1.1", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p2,p0,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id8 = 1.1 or id8 = 33.78", + false, false, true, true, []string{"partition:p0,p1"}, []string{"partition:p2,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id8 >1 AND id8 < 4", + false, false, true, true, []string{"partition:all"}, []string{}, 3, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id9) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:all"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id9 = 46.48", + false, false, true, true, []string{}, []string{"partition:all"}, 0, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id9 = 46.48 or id9 = 336.78", + false, false, true, true, []string{}, []string{"partition:all"}, 0, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id9 >45 AND id9 < 47", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id10) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:all"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id10 = 46.48", + false, false, true, true, []string{}, []string{}, 0, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id10 = 46.48 or id10 = 336.78", + false, false, true, true, []string{}, []string{}, 0, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id10 >366 AND id10 < 368", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + }, + }, + } + executePartTableCase(t, tk, testCases, createSQL, insertSQLS, dropSQL) + + // partition column is date/time family + createSQL2 := "create table tkey_datetime(\n" + + "id1 DATE not null,\n" + + "id2 TIME not null,\n" + + "id3 DATETIME not null,\n" + + "id4 TIMESTAMP not null,\n" + + "id5 YEAR not null,\n" + + "name varchar(20),\n" + + "primary key(id1, id2, id3, id4, id5)\n" + + ")\n" + dropSQL2 := "drop table tkey_datetime" + insertSQLS2 := []string{ + "insert into tkey_datetime values('2012-04-10', '12:12:12', '2012-04-10 12:12:12', '2012-04-10 12:12:12.12345', 2012, 'linpin')", + "insert into tkey_datetime values('2013-05-11', '13:13:13', '2013-05-11 13:13:13', '2013-05-11 13:13:13.43133', 2013, 'minghua')", + "insert into tkey_datetime values('2014-06-12', '14:14:14', '2014-06-12 14:14:14', '2014-06-12 14:14:14.32344', 2014, 'oyangfeng')", + "insert into tkey_datetime values('2015-07-13', '15:15:15', '2015-07-13 15:15:15', '2015-07-13 15:15:15.42544', 2015, 'pengdehuai')", + "insert into tkey_datetime values('2021-08-14', '16:16:16', '2021-08-14 16:16:16', '2021-08-14 16:16:16.18945', 2021, 'shenwanshan')", + "insert into tkey_datetime values('2022-12-23', '23:12:15', '2022-12-23 23:12:15', '2022-12-23 23:12:15.43133', 2022, 'tangchap')", + "insert into tkey_datetime values('2023-01-12', '20:38:14', '2023-01-12 20:38:14', '2023-01-12 20:38:14.32344', 2023, 'xinyu')", + "insert into tkey_datetime values('2018-07-13', '07:15:15', '2018-07-13 07:15:15', '2018-07-13 07:15:15.42544', 2018, 'zongyang')", + "insert into tkey_datetime values('1980-01-30', '00:12:15', '1980-01-30 00:12:15', '1980-01-30 00:12:15.42544', 1980, 'MAYUWEI')", + "insert into tkey_datetime values('1980-03-30', '00:13:15', '1980-03-30 00:13:15', '1980-03-30 00:13:15.42544', 1980, 'maqinwei')", + } + testCases2 := []partTableCase{ + { + partitionbySQL: "PARTITION BY KEY(id1) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_datetime", + false, false, true, true, []string{"partition:all"}, []string{}, 10, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id1 = '2012-04-10'", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id1 = '2012-04-10' or id1 = '2018-07-13'", + false, false, true, true, []string{"partition:p0,p2"}, []string{"partition:p1,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id1 >'2012-04-10' AND id1 < '2014-04-10'", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id3) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_datetime", + false, false, true, true, []string{"partition:all"}, []string{}, 10, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id3 = '2012-04-10 12:12:12'", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id3 = '2012-04-10 12:12:12' or id3 = '2021-08-14 16:16:16'", + false, false, true, true, []string{"partition:p1,p3"}, []string{"partition:p2,p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id3 >'2012-04-10 12:12:12' AND id3 < '2014-04-10 12:12:12'", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id4) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_datetime", + false, false, true, true, []string{"partition:all"}, []string{}, 10, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id4 = '2012-04-10 12:12:12'", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id4 = '2012-04-10 12:12:12' or id4 = '2021-08-14 16:16:16'", + false, false, true, true, []string{"partition:p1,p3"}, []string{"partition:p0,p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id4 >'2012-04-10 12:12:12' AND id4 < '2014-04-10 12:12:12'", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id5) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_datetime", + false, false, true, true, []string{"partition:all"}, []string{}, 10, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id5 = 2012", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id5 = 2012 or id5 = 2018", + false, false, true, true, []string{"partition:p0,p2"}, []string{"partition:p1,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id5 >2012 AND id5 < 2014", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p3,p0"}, 1, + }, + }, + }, + } + executePartTableCase(t, tk, testCases2, createSQL2, insertSQLS2, dropSQL2) + + // partition column is string family + createSQL3 := "create table tkey_string(\n" + + "id1 CHAR(16) not null,\n" + + "id2 VARCHAR(16) not null,\n" + + "id3 BINARY(16) not null,\n" + + "id4 VARBINARY(16) not null,\n" + + "id5 BLOB not null,\n" + + "id6 TEXT not null,\n" + + "id7 ENUM('x-small', 'small', 'medium', 'large', 'x-large') not null,\n" + + "id8 SET ('a', 'b', 'c', 'd') not null,\n" + + "name varchar(16),\n" + + "primary key(id1, id2, id3, id4, id7, id8)\n" + + ")\n" + dropSQL3 := "drop table tkey_string" + insertSQLS3 := []string{ + "INSERT INTO tkey_string VALUES('huaian','huaian','huaian','huaian','huaian','huaian','x-small','a','linpin')", + "INSERT INTO tkey_string VALUES('nanjing','nanjing','nanjing','nanjing','nanjing','nanjing','small','b','linpin')", + "INSERT INTO tkey_string VALUES('zhenjiang','zhenjiang','zhenjiang','zhenjiang','zhenjiang','zhenjiang','medium','c','linpin')", + "INSERT INTO tkey_string VALUES('suzhou','suzhou','suzhou','suzhou','suzhou','suzhou','large','d','linpin')", + "INSERT INTO tkey_string VALUES('wuxi','wuxi','wuxi','wuxi','wuxi','wuxi','x-large','a','linpin')", + } + testCases3 := []partTableCase{ + { + partitionbySQL: "PARTITION BY KEY(id1) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:all"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id1 = 'huaian'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p0,p2"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id1 = 'huaian' or id1 = 'suzhou'", + false, false, true, true, []string{"partition:p0,p3"}, []string{"partition:p1,p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id1 >'huaian' AND id1 < 'suzhou'", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id2) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:all"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id2 = 'huaian'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id2 = 'huaian' or id2 = 'suzhou'", + false, false, true, true, []string{"partition:p0,p3"}, []string{"partition:p1,p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id2 >'huaian' AND id2 < 'suzhou'", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id3) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:all"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id3 = 0x73757A686F7500000000000000000000", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id3 = 0x73757A686F7500000000000000000000 or id3 = 0x6E616E6A696E67000000000000000000", + false, false, true, true, []string{"partition:p0,p1"}, []string{"partition:p2,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id3 >0x67756169616E00000000000000000000 AND id3 < 0x6E616E6A696E67000000000000000000", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id4) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:all"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id4 = 0x68756169616E", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p0,p2"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id4 = 0x68756169616E or id4 = 0x73757A686F75", + false, false, true, true, []string{"partition:p0,p3"}, []string{"partition:p1,p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id4 >0x73757A686F75 AND id4 < 0x78757869", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id7) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:all"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id7 = 'x-small'", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id7 = 'x-small' or id7 = 'large'", + false, false, true, true, []string{"partition:p0,p2"}, []string{"partition:p1,p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id7 > 'large' AND id7 < 'x-small'", + false, false, true, true, []string{"partition:p0,p1,p3"}, []string{"partition:p2"}, 3, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id8) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:all"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0,p2,p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1,p0,p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id8 = 'a'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1,p2,p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id8 = 'a' or id8 = 'b'", + false, false, true, true, []string{"partition:p1,p3"}, []string{"partition:p0,p2"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id8 > 'a' AND id8 < 'c'", + false, false, true, true, []string{"partition:all"}, []string{}, 1, + }, + }, + }, + } + executePartTableCase(t, tk, testCases3, createSQL3, insertSQLS3, dropSQL3) +} + +func TestPruneModeWarningInfo(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustExec("set session tidb_partition_prune_mode = 'dynamic'") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows("Warning 1105 Please analyze all partition tables again for consistency between partition and global stats", + "Warning 1105 Please avoid setting partition prune mode to dynamic at session level and set partition prune mode to dynamic at global level")) + tk.MustExec("set global tidb_partition_prune_mode = 'dynamic'") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 Please analyze all partition tables again for consistency between partition and global stats")) +} + +func TestPartitionByIntListExtensivePart(t *testing.T) { + limitSizeOfTest := true + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "PartitionByIntListExtensive" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use " + schemaName) + + tBase := `(lp tinyint unsigned, a int unsigned, b varchar(255) collate utf8mb4_general_ci, c int, d datetime, e timestamp, f double, g text, key (b), key (c,b), key (d,c), key(e), primary key (a, lp))` + t2Str := `create table t2 ` + tBase + tStr := `create table t ` + tBase + + rows := 100 + pkInserts := 20 + pkUpdates := 20 + pkDeletes := 10 // Enough to delete half of what is inserted? + tStart := []string{ + // Non partitioned + tStr, + // RANGE COLUMNS + tStr + ` partition by list (lp) (partition p0 values in (0,6),partition p1 values in (1), partition p2 values in (2), partition p3 values in (3), partition p4 values in (4,5))`, + // KEY + tStr + ` partition by key(a) partitions 5`, + // HASH + tStr + ` partition by hash(a) partitions 5`, + // HASH with function + tStr + ` partition by hash(a DIV 3) partitions 5`, + } + if limitSizeOfTest { + tStart = tStart[:2] + } + quarterUintRange := 1 << 30 + quarterUintRangeStr := fmt.Sprintf("%d", quarterUintRange) + halfUintRangeStr := fmt.Sprintf("%d", 2*quarterUintRange) + threeQuarterUintRangeStr := fmt.Sprintf("%d", 3*quarterUintRange) + tAlter := []string{ + // LIST + `alter table t partition by list (lp) (partition p0 values in (2), partition p1 values in (1,3,5), partition p2 values in (0,4,6))`, + `alter table t partition by list (lp) (partition p3 values in (3), partition p4 values in (4), partition p2 values in (2), partition p6 values in (6), partition p5 values in (5), partition p1 values in (1), partition p0 values in (0))`, + // LIST COLUMNS + `alter table t partition by list columns (lp) (partition p0 values in (2), partition p1 values in (1,3,5), partition p2 values in (0,4,6))`, + `alter table t partition by list columns (lp) (partition p3 values in (3), partition p4 values in (4), partition p2 values in (2), partition p6 values in (6), partition p5 values in (5), partition p1 values in (1), partition p0 values in (0))`, + // RANGE COLUMNS + `alter table t partition by range (a) (partition pFirst values less than (` + halfUintRangeStr + `), partition pLast values less than (MAXVALUE))`, + // RANGE COLUMNS + `alter table t partition by range (a) (partition pFirst values less than (` + quarterUintRangeStr + `),` + + `partition pLowMid values less than (` + halfUintRangeStr + `),` + + `partition pHighMid values less than (` + threeQuarterUintRangeStr + `),` + + `partition pLast values less than (maxvalue))`, + // KEY + `alter table t partition by key(a) partitions 7`, + `alter table t partition by key(a) partitions 3`, + // Hash + `alter table t partition by hash(a) partitions 7`, + `alter table t partition by hash(a) partitions 3`, + // Hash + `alter table t partition by hash(a DIV 13) partitions 7`, + `alter table t partition by hash(a DIV 13) partitions 3`, + } + if limitSizeOfTest { + tAlter = tAlter[:2] + } + + seed := gotime.Now().UnixNano() + logutil.BgLogger().Info("Seeding rand", zap.Int64("seed", seed)) + reorgRand := rand.New(rand.NewSource(seed)) + for _, createSQL := range tStart { + for _, alterSQL := range tAlter { + tk.MustExec(createSQL) + tk.MustExec(t2Str) + getNewPK := getNewIntPK() + getValues := getInt7ValuesFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, alterSQL, rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) + tk.MustExec(`drop table t`) + tk.MustExec(`drop table t2`) + } + } + for _, createSQL := range tStart[1:] { + tk.MustExec(createSQL) + tk.MustExec(t2Str) + getNewPK := getNewIntPK() + getValues := getInt7ValuesFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, "alter table t remove partitioning", rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) + tk.MustExec(`drop table t`) + tk.MustExec(`drop table t2`) + } +} + +func getInt7ValuesFunc() func(string, bool, *rand.Rand) string { + cnt := 0 + return func(pk string, asAssignment bool, reorgRand *rand.Rand) string { + s := `(%d, %s, '%s', %d, '%s', '%s', %f, '%s')` + if asAssignment { + s = `lp = %d, a = %s, b = '%s', c = %d, d = '%s', e = '%s', f = %f, g = '%s'` + } + cnt++ + lp, err := strconv.Atoi(pk) + if err != nil { + lp = 0 + } + return fmt.Sprintf(s, + lp%7, + pk, + randStr(reorgRand.Intn(19), reorgRand), + cnt, //reorgRand.Int31(), + gotime.Unix(413487608+int64(reorgRand.Intn(1705689644)), 0).Format("2006-01-02T15:04:05"), + gotime.Unix(413487608+int64(reorgRand.Intn(1705689644)), 0).Format("2006-01-02T15:04:05"), + reorgRand.Float64(), + randStr(512+reorgRand.Intn(1024), reorgRand)) + } +} + +func TestPartitionByIntExtensivePart(t *testing.T) { + limitSizeOfTest := true + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "PartitionByIntExtensive" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use " + schemaName) + + tBase := `(a int unsigned, b varchar(255) collate utf8mb4_general_ci, c int, d datetime, e timestamp, f double, g text, primary key (a), key (b), key (c,b), key (d,c), key(e))` + t2Str := `create table t2 ` + tBase + tStr := `create table t ` + tBase + + rows := 100 + pkInserts := 20 + pkUpdates := 20 + pkDeletes := 10 // Enough to delete half of what is inserted? + thirdUintRange := 1 << 32 / 2 + thirdUintRangeStr := fmt.Sprintf("%d", thirdUintRange) + twoThirdUintRangeStr := fmt.Sprintf("%d", 2*thirdUintRange) + tStart := []string{ + // Non partitioned + tStr, + // RANGE COLUMNS + tStr + ` partition by range (a) (partition pFirst values less than (` + thirdUintRangeStr + `),` + + `partition pMid values less than (` + twoThirdUintRangeStr + `), partition pLast values less than (maxvalue))`, + // KEY + tStr + ` partition by key(a) partitions 5`, + // HASH + tStr + ` partition by hash(a) partitions 5`, + // HASH with function + tStr + ` partition by hash(a DIV 3) partitions 5`, + } + if limitSizeOfTest { + tStart = tStart[:2] + } + quarterUintRange := 1 << 30 + quarterUintRangeStr := fmt.Sprintf("%d", quarterUintRange) + halfUintRangeStr := fmt.Sprintf("%d", 2*quarterUintRange) + threeQuarterUintRangeStr := fmt.Sprintf("%d", 3*quarterUintRange) + tAlter := []string{ + // RANGE COLUMNS + `alter table t partition by range (a) (partition pFirst values less than (` + halfUintRangeStr + `), partition pLast values less than (MAXVALUE))`, + // RANGE COLUMNS + `alter table t partition by range (a) (partition pFirst values less than (` + quarterUintRangeStr + `),` + + `partition pLowMid values less than (` + halfUintRangeStr + `),` + + `partition pHighMid values less than (` + threeQuarterUintRangeStr + `),` + + `partition pLast values less than (maxvalue))`, + // KEY + `alter table t partition by key(a) partitions 7`, + `alter table t partition by key(a) partitions 3`, + // Hash + `alter table t partition by hash(a) partitions 7`, + `alter table t partition by hash(a) partitions 3`, + // Hash + `alter table t partition by hash(a DIV 13) partitions 7`, + `alter table t partition by hash(a DIV 13) partitions 3`, + } + if limitSizeOfTest { + tAlter = tAlter[:2] + } + + seed := gotime.Now().UnixNano() + logutil.BgLogger().Info("Seeding rand", zap.Int64("seed", seed)) + reorgRand := rand.New(rand.NewSource(seed)) + for _, createSQL := range tStart { + for _, alterSQL := range tAlter { + tk.MustExec(createSQL) + tk.MustExec(t2Str) + getNewPK := getNewIntPK() + getValues := getIntValuesFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, alterSQL, rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) + tk.MustExec(`drop table t`) + tk.MustExec(`drop table t2`) + } + } + for _, createSQL := range tStart[1:] { + tk.MustExec(createSQL) + tk.MustExec(t2Str) + getNewPK := getNewIntPK() + getValues := getIntValuesFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, "alter table t remove partitioning", rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) + tk.MustExec(`drop table t`) + tk.MustExec(`drop table t2`) + } +} + +func TestGlobalIndexPartitionByIntExtensivePart(t *testing.T) { + limitSizeOfTest := true + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "PartitionByIntExtensive" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk.MustExec(`set @@tidb_enable_global_index = ON`) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use " + schemaName) + tk2.MustExec(`set @@tidb_enable_global_index = ON`) + + tBase := `(a int unsigned not null, b varchar(255) collate utf8mb4_general_ci, c int, d datetime, e timestamp, f double, g text, unique key idx_a(a), unique key idx_b(b), key (c,b), unique key idx_dc(d,c), key(e))` + tBaseA := `(a int unsigned not null, b varchar(255) collate utf8mb4_general_ci, c int, d datetime, e timestamp, f double, g text, unique key idx_a(a), unique key idx_b(b) Global, key (c,b), unique key idx_dc(d,c) Global, key(e))` + tBaseB := `(a int unsigned not null, b varchar(255) collate utf8mb4_general_ci, c int, d datetime, e timestamp, f double, g text, unique key idx_a(a) Global, unique key idx_b(b), key (c,b), unique key idx_dc(d,c) Global, key(e))` + t2Str := `create table t2 ` + tBase + tStr := `create table t ` + + rows := 100 + pkInserts := 20 + pkUpdates := 20 + pkDeletes := 10 // Enough to delete half of what is inserted? + thirdUintRange := 1 << 32 / 2 + thirdUintRangeStr := fmt.Sprintf("%d", thirdUintRange) + twoThirdUintRangeStr := fmt.Sprintf("%d", 2*thirdUintRange) + tStart := []string{ + // Non partitioned + tStr + tBase, + // RANGE COLUMNS + tStr + tBaseA + ` partition by range (a) (partition pFirst values less than (` + thirdUintRangeStr + `),` + + `partition pMid values less than (` + twoThirdUintRangeStr + `), partition pLast values less than (maxvalue))`, + // KEY + tStr + tBaseB + ` partition by key(b) partitions 5`, + // HASH + tStr + tBaseA + ` partition by hash(a) partitions 5`, + // HASH with function + tStr + tBaseA + ` partition by hash(a DIV 3) partitions 5`, + } + if limitSizeOfTest { + tStart = tStart[:2] + } + quarterUintRange := 1 << 30 + quarterUintRangeStr := fmt.Sprintf("%d", quarterUintRange) + halfUintRangeStr := fmt.Sprintf("%d", 2*quarterUintRange) + threeQuarterUintRangeStr := fmt.Sprintf("%d", 3*quarterUintRange) + tAlter := []string{ + // RANGE COLUMNS + //`alter table t partition by range columns (b) (partition pFirst values less than ("m"), partition pLast values less than (MAXVALUE))`, + // RANGE COLUMNS + `alter table t partition by range (a+2) (partition pFirst values less than (` + quarterUintRangeStr + `),` + + `partition pLowMid values less than (` + halfUintRangeStr + `),` + + `partition pHighMid values less than (` + threeQuarterUintRangeStr + `),` + + `partition pLast values less than (maxvalue)) update indexes (idx_a local, idx_dc global, idx_b global)`, + // KEY + `alter table t partition by key(b) partitions 3 update indexes(idx_a global, idx_b local, idx_dc global)`, + // Hash + `alter table t partition by hash(a) partitions 7 update indexes (idx_dc global, idx_a local, idx_b global)`, + } + if limitSizeOfTest { + tAlter = tAlter[:2] + } + + seed := gotime.Now().UnixNano() + logutil.BgLogger().Info("Seeding rand", zap.Int64("seed", seed)) + reorgRand := rand.New(rand.NewSource(seed)) + for _, createSQL := range tStart { + for _, alterSQL := range tAlter { + tk.MustExec(createSQL) + tk.MustExec(t2Str) + getNewPK := getNewIntPK() + getValues := getIntValuesUniqueFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, alterSQL, rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) + tk.MustExec(`drop table t`) + tk.MustExec(`drop table t2`) + } + } + for _, createSQL := range tStart[1:] { + tk.MustExec(createSQL) + tk.MustExec(t2Str) + getNewPK := getNewIntPK() + getValues := getIntValuesUniqueFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, "alter table t remove partitioning", rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) + tk.MustExec(`drop table t`) + tk.MustExec(`drop table t2`) + } +} + +func getNewIntPK() func(map[string]struct{}, string, *rand.Rand) string { + return func(m map[string]struct{}, suf string, reorgRand *rand.Rand) string { + uintPK := reorgRand.Uint32() + newPK := strconv.FormatUint(uint64(uintPK), 10) + for _, ok := m[newPK]; ok; { + uintPK = reorgRand.Uint32() + newPK = strconv.FormatUint(uint64(uintPK), 10) + _, ok = m[newPK] + } + m[newPK] = struct{}{} + return newPK + } +} + +func getIntValuesFunc() func(string, bool, *rand.Rand) string { + cnt := 0 + return func(pk string, asAssignment bool, reorgRand *rand.Rand) string { + s := `(%s, '%s', %d, '%s', '%s', %f, '%s')` + if asAssignment { + s = `a = %s, b = '%s', c = %d, d = '%s', e = '%s', f = %f, g = '%s'` + } + cnt++ + return fmt.Sprintf(s, + pk, + randStr(reorgRand.Intn(19), reorgRand), + cnt, //reorgRand.Int31(), + gotime.Unix(413487608+int64(reorgRand.Intn(1705689644)), 0).Format("2006-01-02T15:04:05"), + gotime.Unix(413487608+int64(reorgRand.Intn(1705689644)), 0).Format("2006-01-02T15:04:05"), + reorgRand.Float64(), + randStr(512+reorgRand.Intn(1024), reorgRand)) + } +} + +func getIntValuesUniqueFunc() func(string, bool, *rand.Rand) string { + cnt := 0 + return func(pk string, asAssignment bool, reorgRand *rand.Rand) string { + s := `(%s, '%s', %d, '%s', '%s', %f, '%s')` + if asAssignment { + s = `a = %s, b = '%s', c = %d, d = '%s', e = '%s', f = %f, g = '%s'` + } + cnt++ + return fmt.Sprintf(s, + pk, + randStr(reorgRand.Intn(19), reorgRand)+pk, + cnt, //reorgRand.Int31(), + gotime.Unix(413487608+int64(reorgRand.Intn(1705689644)), 0).Format("2006-01-02T15:04:05"), + gotime.Unix(413487608+int64(reorgRand.Intn(1705689644)), 0).Format("2006-01-02T15:04:05"), + reorgRand.Float64(), + randStr(512+reorgRand.Intn(1024), reorgRand)) + } +} + +func TestPartitionByExtensivePart(t *testing.T) { + limitSizeOfTest := true + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "PartitionByExtensive" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use " + schemaName) + + tBase := `(a varchar(255) collate utf8mb4_unicode_ci, b varchar(255) collate utf8mb4_general_ci, c int, d datetime, e timestamp, f double, g text, primary key (a), key (b), key (c,b), key (d,c), key(e))` + t2Str := `create table t2 ` + tBase + tStr := `create table t ` + tBase + + rows := 100 + pkInserts := 20 + pkUpdates := 20 + pkDeletes := 10 // Enough to delete half of what is inserted? + tStart := []string{ + // Non partitioned + tStr, + // RANGE COLUMNS + tStr + ` partition by range columns (a) (partition pNull values less than (""), partition pM values less than ("M"), partition pLast values less than (maxvalue))`, + // KEY + tStr + ` partition by key(a) partitions 5`, + } + if limitSizeOfTest { + tStart = tStart[:2] + } + showCreateStr := "t CREATE TABLE `t` (\n" + + " `a` varchar(255) COLLATE utf8mb4_unicode_ci NOT NULL,\n" + + " `b` varchar(255) COLLATE utf8mb4_general_ci DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " `d` datetime DEFAULT NULL,\n" + + " `e` timestamp NULL DEFAULT NULL,\n" + + " `f` double DEFAULT NULL,\n" + + " `g` text DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`),\n" + + " KEY `c` (`c`,`b`),\n" + + " KEY `d` (`d`,`c`),\n" + + " KEY `e` (`e`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + tAlter := []struct{ alter, result string }{ + { + // RANGE COLUMNS + alter: `alter table t partition by range columns (a) (partition pH values less than ("H"), partition pLast values less than (MAXVALUE))`, + result: showCreateStr + + "PARTITION BY RANGE COLUMNS(`a`)\n" + + "(PARTITION `pH` VALUES LESS THAN ('H'),\n" + + " PARTITION `pLast` VALUES LESS THAN (MAXVALUE))", + }, + { + // RANGE COLUMNS + alter: `alter table t partition by range columns (a) (partition pNull values less than (""), partition pG values less than ("G"), partition pR values less than ("R"), partition pLast values less than (maxvalue))`, + result: showCreateStr + + "PARTITION BY RANGE COLUMNS(`a`)\n" + + "(PARTITION `pNull` VALUES LESS THAN (''),\n" + + " PARTITION `pG` VALUES LESS THAN ('G'),\n" + + " PARTITION `pR` VALUES LESS THAN ('R'),\n" + + " PARTITION `pLast` VALUES LESS THAN (MAXVALUE))", + }, + // KEY + { + alter: `alter table t partition by key(a) partitions 7`, + result: showCreateStr + + "PARTITION BY KEY (`a`) PARTITIONS 7", + }, + { + alter: `alter table t partition by key(a) partitions 3`, + result: showCreateStr + + "PARTITION BY KEY (`a`) PARTITIONS 3", + }, + } + if limitSizeOfTest { + tAlter = tAlter[:2] + } + + seed := gotime.Now().UnixNano() + logutil.BgLogger().Info("Seeding rand", zap.Int64("seed", seed)) + reorgRand := rand.New(rand.NewSource(seed)) + for _, createSQL := range tStart { + for _, alterSQL := range tAlter { + tk.MustExec(createSQL) + tk.MustExec(t2Str) + getNewPK := getNewStringPK() + getValues := getValuesFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, alterSQL.alter, rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) + res := tk.MustQuery(`show create table t`) + res.AddComment("create SQL: " + createSQL + "\nalterSQL: " + alterSQL.alter) + res.Check(testkit.Rows(alterSQL.result)) + tk.MustExec(`drop table t`) + tk.MustExec(`drop table t2`) + } + } + + for _, createSQL := range tStart[1:] { + tk.MustExec(createSQL) + tk.MustExec(t2Str) + getNewPK := getNewStringPK() + getValues := getValuesFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, "alter table t remove partitioning", rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) + tk.MustExec(`drop table t`) + tk.MustExec(`drop table t2`) + } +} + +func TestReorgPartExtensivePart(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "ReorgPartExtensive" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use " + schemaName) + // TODO: Handle different column types? + // TODO: Handle index for different column types / combinations as well? + + // Steps: + // - create a table (should at least test both LIST and RANGE partition, Including COLUMNS) + // - add base data + // - start DDL + // - before each (and after?) each state transition: + // - insert, update and delete concurrently, to verify that the states are correctly handled. + // - TODO: Crash (if rollback is needed, then OK, but the rest need to be tested + // - TODO: Fail + // - TODO: run queries that could clash with backfill etc. (How to handle expected errors?) + // - TODO: on both the 'current' state and 'previous' state! + // - run ADMIN CHECK TABLE + // + + tk.MustExec(`create table t (a varchar(255) collate utf8mb4_unicode_ci, b varchar(255) collate utf8mb4_general_ci, c int, d datetime, e timestamp, f double, g text, primary key (a)) partition by range columns (a) (partition pNull values less than (""), partition pM values less than ("M"), partition pLast values less than (maxvalue))`) + tk.MustExec(`create table t2 (a varchar(255) collate utf8mb4_unicode_ci, b varchar(255) collate utf8mb4_general_ci, c int, d datetime, e timestamp, f double, g text, primary key (a), key (b), key (c,b), key (d,c), key(e))`) + + // TODO: Test again with timestamp in col e!! + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` varchar(255) COLLATE utf8mb4_unicode_ci NOT NULL,\n" + + " `b` varchar(255) COLLATE utf8mb4_general_ci DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " `d` datetime DEFAULT NULL,\n" + + " `e` timestamp NULL DEFAULT NULL,\n" + + " `f` double DEFAULT NULL,\n" + + " `g` text DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE COLUMNS(`a`)\n" + + "(PARTITION `pNull` VALUES LESS THAN (''),\n" + + " PARTITION `pM` VALUES LESS THAN ('M'),\n" + + " PARTITION `pLast` VALUES LESS THAN (MAXVALUE))")) + + tk.MustQuery(`show create table t2`).Check(testkit.Rows("" + + "t2 CREATE TABLE `t2` (\n" + + " `a` varchar(255) COLLATE utf8mb4_unicode_ci NOT NULL,\n" + + " `b` varchar(255) COLLATE utf8mb4_general_ci DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " `d` datetime DEFAULT NULL,\n" + + " `e` timestamp NULL DEFAULT NULL,\n" + + " `f` double DEFAULT NULL,\n" + + " `g` text DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`),\n" + + " KEY `c` (`c`,`b`),\n" + + " KEY `d` (`d`,`c`),\n" + + " KEY `e` (`e`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + + rows := 1000 + pkInserts := 200 + pkUpdates := 200 + pkDeletes := 100 // Enough to delete half of what is inserted? + alterStr := `alter table t reorganize partition pNull, pM, pLast into (partition pI values less than ("I"), partition pQ values less than ("q"), partition pLast values less than (MAXVALUE))` + seed := rand.Int63() + logutil.BgLogger().Info("Seeding rand", zap.Int64("seed", seed)) + reorgRand := rand.New(rand.NewSource(seed)) + getNewPK := getNewStringPK() + getValues := getValuesFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, alterStr, rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) +} + +func getNewStringPK() func(map[string]struct{}, string, *rand.Rand) string { + return func(m map[string]struct{}, suf string, reorgRand *rand.Rand) string { + newPK := randStr(2+reorgRand.Intn(5), reorgRand) + suf + lowerPK := strings.ToLower(newPK) + for _, ok := m[lowerPK]; ok; { + newPK = randStr(2+reorgRand.Intn(5), reorgRand) + lowerPK = strings.ToLower(newPK) + _, ok = m[lowerPK] + } + m[lowerPK] = struct{}{} + return newPK + } +} + +func getValuesFunc() func(string, bool, *rand.Rand) string { + cnt := 0 + return func(pk string, asAssignment bool, reorgRand *rand.Rand) string { + s := `('%s', '%s', %d, '%s', '%s', %f, '%s')` + if asAssignment { + s = `a = '%s', b = '%s', c = %d, d = '%s', e = '%s', f = %f, g = '%s'` + } + cnt++ + return fmt.Sprintf(s, + pk, + randStr(reorgRand.Intn(19), reorgRand), + cnt, //reorgRand.Int31(), + gotime.Unix(413487608+int64(reorgRand.Intn(1705689644)), 0).Format("2006-01-02T15:04:05"), + gotime.Unix(413487608+int64(reorgRand.Intn(1705689644)), 0).Format("2006-01-02T15:04:05"), + reorgRand.Float64(), + randStr(512+reorgRand.Intn(1024), reorgRand)) + } +} + +func checkDMLInAllStates(t *testing.T, tk, tk2 *testkit.TestKit, schemaName, alterStr string, + rows, pkInserts, pkUpdates, pkDeletes int, + reorgRand *rand.Rand, + getNewPK func(map[string]struct{}, string, *rand.Rand) string, + getValues func(string, bool, *rand.Rand) string, +) { + pkMap := make(map[string]struct{}, rows) + pkArray := make([]string, 0, len(pkMap)) + // Generate a start set: + for i := 0; i < rows; i++ { + pk := getNewPK(pkMap, "-o", reorgRand) + pkArray = append(pkArray, pk) + values := getValues(pk, false, reorgRand) + tk.MustExec(`insert into t values ` + values) + tk.MustExec(`insert into t2 values ` + values) + } + tk.MustExec(`analyze table t`) + tk.MustExec(`analyze table t2`) + tk.MustQuery(`select * from t except select * from t2`).Check(testkit.Rows()) + tk.MustQuery(`select * from t2 except select * from t`).Check(testkit.Rows()) + + // How to arrange data for possible collisions? + // change both PK column, SK column and non indexed column! + // Run various changes in transactions, in two concurrent sessions + // + mirror those transactions on a copy of the same table and data without DDL + // to verify expected transaction conflicts! + // We should try to collide: + // Current data : 1-1000 + // insert vN 1-200 // random order, random length of transaction? + // insert vN-1 100-300 // interleaved with vN, random order+length of txn? + // update vN 1-20, 100-120, 200-220, 300-320.. + // update vN-1 10-30, 110-130, 210-230, ... + // delete vN + // delete vN-1 + // insert update delete <- + // insert + // update + // delete + // Note: update the PK so it moves between different before and after partitions + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + currentState := model.StateNone + transitions := 0 + var currTbl table.Table + currSchema := sessiontxn.GetTxnManager(tk2.Session()).GetTxnInfoSchema() + prevTbl, err := currSchema.TableByName(context.Background(), model.NewCIStr(schemaName), model.NewCIStr("t")) + require.NoError(t, err) + var hookErr error + testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/onJobRunBefore", func(job *model.Job) { + if hookErr != nil { + // Enough to find a single error + return + } + if job.Type == model.ActionReorganizePartition && job.SchemaState != currentState { + transitions++ + // use random generation to possibly trigger txn collisions / deadlocks? + // insert (dup in new/old , non dup) + // update (dup in new/old , non dup as in same old/new partition -> update, different new/old -> insert + delete) + // delete + // verify with select after commit? + + logutil.BgLogger().Info("State before ins/upd/del", zap.Int("transitions", transitions), + zap.Int("rows", len(pkMap)), zap.Stringer("SchemaState", job.SchemaState)) + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + // Start with PK changes (non duplicate keys) + insPK := make([]string, 0, pkInserts) + values := make([]string, 0, pkInserts) + for i := 0; i < pkInserts; i += 2 { + pk := getNewPK(pkMap, "-i0", reorgRand) + logutil.BgLogger().Debug("insert1", zap.String("pk", pk)) + pkArray = append(pkArray, pk) + insPK = append(insPK, pk) + values = append(values, getValues(pk, false, reorgRand)) + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + hookErr = tk2.ExecToErr(`insert into t values ` + strings.Join(values, ",")) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`insert into t2 values ` + strings.Join(values, ",")) + if hookErr != nil { + return + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + currSchema = sessiontxn.GetTxnManager(tk2.Session()).GetTxnInfoSchema() + currTbl, hookErr = currSchema.TableByName(context.Background(), model.NewCIStr(schemaName), model.NewCIStr("t")) + + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using previous schema version + + values = values[:0] + for i := 1; i < pkInserts; i += 2 { + pk := getNewPK(pkMap, "-i1", reorgRand) + logutil.BgLogger().Debug("insert2", zap.String("pk", pk)) + pkArray = append(pkArray, pk) + insPK = append(insPK, pk) + values = append(values, getValues(pk, false, reorgRand)) + } + hookErr = tk2.ExecToErr(`insert into t values ` + strings.Join(values, ",")) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`insert into t2 values ` + strings.Join(values, ",")) + if hookErr != nil { + return + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + rs, err := tk2.Exec(`select count(*) from t`) + if err != nil { + hookErr = err + return + } + tRows := tk2.ResultSetToResult(rs, "").Rows()[0][0].(string) + rs, err = tk2.Exec(`select count(*) from t2`) + if err != nil { + hookErr = err + return + } + t2Rows := tk2.ResultSetToResult(rs, "").Rows()[0][0].(string) + if tRows != t2Rows { + logutil.BgLogger().Error("rows do not match", zap.String("t", tRows), zap.String("t2", t2Rows), zap.Stringer("state", job.SchemaState)) + } + + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using current schema version + + // Half from insert (1/4 in current schema version) + values = values[:0] + for i := 0; i < pkUpdates; i += 4 { + insIdx := reorgRand.Intn(len(insPK)) + oldPK := insPK[insIdx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + newPK := getNewPK(pkMap, "-u0", reorgRand) + insPK[insIdx] = newPK + idx := len(pkArray) - len(insPK) + insIdx + pkArray[idx] = newPK + value := getValues(newPK, true, reorgRand) + + logutil.BgLogger().Debug("update1", zap.String("old", oldPK), zap.String("value", value)) + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + + // Also do some non-pk column updates! + insIdx = reorgRand.Intn(len(insPK)) + oldPK = insPK[insIdx] + value = getValues(oldPK, true, reorgRand) + + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using previous schema version + + // Half from insert (1/4 in previous schema version) + values = values[:0] + for i := 1; i < pkUpdates; i += 4 { + insIdx := reorgRand.Intn(len(insPK)) + oldPK := insPK[insIdx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + newPK := getNewPK(pkMap, "-u1", reorgRand) + insPK[insIdx] = newPK + idx := len(pkArray) - len(insPK) + insIdx + pkArray[idx] = newPK + value := getValues(newPK, true, reorgRand) + logutil.BgLogger().Debug("update2", zap.String("old", oldPK), zap.String("value", value)) + + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + + // Also do some non-pk column updates! + // Note: if PK changes it does RemoveRecord + AddRecord + insIdx = reorgRand.Intn(len(insPK)) + oldPK = insPK[insIdx] + value = getValues(oldPK, true, reorgRand) + + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + // Half from Old + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using current schema version + + // Half from old (1/4 in current schema version) + values = values[:0] + for i := 2; i < pkUpdates; i += 4 { + idx := reorgRand.Intn(len(pkArray) - len(insPK)) + oldPK := pkArray[idx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + newPK := getNewPK(pkMap, "-u2", reorgRand) + pkArray[idx] = newPK + value := getValues(newPK, true, reorgRand) + logutil.BgLogger().Debug("update3", zap.String("old", oldPK), zap.String("value", value)) + + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + + // Also do some non-pk column updates! + idx = reorgRand.Intn(len(pkArray) - len(insPK)) + oldPK = pkArray[idx] + value = getValues(oldPK, true, reorgRand) + + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using previous schema version + + // Half from old (1/4 in previous schema version) + values = values[:0] + for i := 3; i < pkUpdates; i += 4 { + idx := reorgRand.Intn(len(pkArray) - len(insPK)) + oldPK := pkArray[idx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + newPK := getNewPK(pkMap, "-u3", reorgRand) + pkArray[idx] = newPK + value := getValues(newPK, true, reorgRand) + logutil.BgLogger().Debug("update4", zap.String("old", oldPK), zap.String("value", value)) + + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + + // Also do some non-pk column updates! + idx = reorgRand.Intn(len(pkArray) - len(insPK)) + oldPK = pkArray[idx] + value = getValues(oldPK, true, reorgRand) + + hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`update t2 set ` + value + ` where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + rs, err = tk2.Exec(`select count(*) from t`) + if err != nil { + hookErr = err + return + } + tRows = tk2.ResultSetToResult(rs, "").Rows()[0][0].(string) + rs, err = tk2.Exec(`select count(*) from t2`) + if err != nil { + hookErr = err + return + } + t2Rows = tk2.ResultSetToResult(rs, "").Rows()[0][0].(string) + if tRows != t2Rows { + logutil.BgLogger().Error("rows do not match", zap.String("t", tRows), zap.String("t2", t2Rows), zap.Stringer("state", job.SchemaState)) + } + + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using current schema version + + // Half from insert (1/4 in current schema version) + values = values[:0] + for i := 0; i < pkDeletes; i += 4 { + insIdx := reorgRand.Intn(len(insPK)) + oldPK := insPK[insIdx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + idx := len(pkArray) - len(insPK) + insIdx + insPK = append(insPK[:insIdx], insPK[insIdx+1:]...) + pkArray = append(pkArray[:idx], pkArray[idx+1:]...) + logutil.BgLogger().Debug("delete0", zap.String("pk", oldPK)) + + hookErr = tk2.ExecToErr(`delete from t where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`delete from t2 where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using previous schema version + + // Half from insert (1/4 in previous schema version) + values = values[:0] + for i := 1; i < pkDeletes; i += 4 { + insIdx := reorgRand.Intn(len(insPK)) + oldPK := insPK[insIdx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + idx := len(pkArray) - len(insPK) + insIdx + insPK = append(insPK[:insIdx], insPK[insIdx+1:]...) + pkArray = append(pkArray[:idx], pkArray[idx+1:]...) + logutil.BgLogger().Debug("delete1", zap.String("pk", oldPK)) + + hookErr = tk2.ExecToErr(`delete from t where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`delete from t2 where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + // Half from Old + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using current schema version + + // Half from old (1/4 in current schema version) + values = values[:0] + for i := 2; i < pkDeletes; i += 4 { + idx := reorgRand.Intn(len(pkArray) - len(insPK)) + oldPK := pkArray[idx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + pkArray = append(pkArray[:idx], pkArray[idx+1:]...) + logutil.BgLogger().Debug("delete2", zap.String("pk", oldPK)) + + hookErr = tk2.ExecToErr(`delete from t where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`delete from t2 where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + if len(pkMap) != len(pkArray) { + panic("Different length!!!") + } + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using previous schema version + + // Half from old (1/4 in previous schema version) + values = values[:0] + for i := 3; i < pkDeletes; i += 4 { + idx := reorgRand.Intn(len(pkArray) - len(insPK)) + oldPK := pkArray[idx] + lowerPK := strings.ToLower(oldPK) + delete(pkMap, lowerPK) + pkArray = append(pkArray[:idx], pkArray[idx+1:]...) + logutil.BgLogger().Debug("delete3", zap.String("pk", oldPK)) + + hookErr = tk2.ExecToErr(`delete from t where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + hookErr = tk2.ExecToErr(`delete from t2 where a = "` + oldPK + `"`) + if hookErr != nil { + return + } + } + tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + rs, err = tk2.Exec(`select count(*) from t`) + if err != nil { + hookErr = err + return + } + tRows = tk2.ResultSetToResult(rs, "").Rows()[0][0].(string) + rs, err = tk2.Exec(`select count(*) from t2`) + if err != nil { + hookErr = err + return + } + t2Rows = tk2.ResultSetToResult(rs, "").Rows()[0][0].(string) + if tRows != t2Rows { + logutil.BgLogger().Error("rows do not match", zap.String("t", tRows), zap.String("t2", t2Rows), zap.Stringer("state", job.SchemaState)) + } + + require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) + // Now using current schema version + tk2.MustQuery(`select count(*) from t2`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + tk2.MustQuery(`select count(*) from t`).Check(testkit.Rows(fmt.Sprintf("%d", len(pkMap)))) + prevTbl = currTbl + logutil.BgLogger().Info("State after ins/upd/del", zap.Int("transitions", transitions), + zap.Int("rows", len(pkMap)), zap.Stringer("SchemaState", job.SchemaState)) + } + }) + defer testfailpoint.Disable(t, "github.com/pingcap/tidb/pkg/ddl/onJobRunBefore") + tk.MustExec(alterStr) + require.NoError(t, hookErr) + tk.MustExec(`admin check table t`) + tk.MustExec(`admin check table t2`) + res := tk.MustQuery(`select * from t`) + res2 := tk.MustQuery(`select * from t2`) + require.Equal(t, res.Sort().Rows(), res2.Sort().Rows()) + res = tk.MustQuery(`select * from t order by a`) + res2 = tk.MustQuery(`select * from t2 order by a`) + require.Equal(t, res.Rows(), res2.Rows()) + // b may be empty string and allows different order, so also do additional Sort + res = tk.MustQuery(`select * from t order by b`).Sort() + res2 = tk.MustQuery(`select * from t2 order by b`).Sort() + require.Equal(t, res.Rows(), res2.Rows()) + tk.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) + tk.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) + tk.MustQuery(`select * from t except select * from t2 LIMIT 1`).Check(testkit.Rows()) + tk.MustQuery(`select * from t2 except select * from t LIMIT 1`).Check(testkit.Rows()) +} + +// Emojis fold to a single rune, and ö compares as o, so just complicated having other runes. +// Enough to just distribute between A and Z + testing simple folding +var runes = []rune("abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ") + +func randStr(n int, r *rand.Rand) string { + var sb strings.Builder + sb.Grow(n) + for i := 0; i < n; i++ { + _, _ = sb.WriteRune(runes[r.Intn(len(runes))]) + } + return sb.String() +} + +func TestPointGetKeyPartitioning(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`CREATE TABLE t (a VARCHAR(30) NOT NULL, b VARCHAR(45) NOT NULL, + c VARCHAR(45) NOT NULL, PRIMARY KEY (b, a)) PARTITION BY KEY(b) PARTITIONS 5`) + tk.MustExec(`INSERT INTO t VALUES ('Aa', 'Ab', 'Ac'), ('Ba', 'Bb', 'Bc')`) + tk.MustQuery(`SELECT * FROM t WHERE b = 'Ab'`).Check(testkit.Rows("Aa Ab Ac")) +} + +func TestExplainPartition(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) PARTITION BY hash(a) PARTITIONS 3`) + tk.MustExec(`INSERT INTO t VALUES (1,1),(2,2),(3,3),(4,4),(5,5),(6,6)`) + tk.MustExec(`analyze table t all columns`) + tk.MustExec(`set tidb_partition_prune_mode = 'static'`) + tk.MustQuery(`EXPLAIN FORMAT = 'brief' SELECT * FROM t WHERE a = 3`).Check(testkit.Rows(""+ + `TableReader 1.00 root data:Selection`, + `└─Selection 1.00 cop[tikv] eq(test.t.a, 3)`, + ` └─TableFullScan 2.00 cop[tikv] table:t, partition:p0 keep order:false`)) + tk.MustExec(`set tidb_partition_prune_mode = 'dynamic'`) + tk.MustQuery(`EXPLAIN FORMAT = 'brief' SELECT * FROM t WHERE a = 3`).Check(testkit.Rows(""+ + `TableReader 1.00 root partition:p0 data:Selection`, + `└─Selection 1.00 cop[tikv] eq(test.t.a, 3)`, + ` └─TableFullScan 6.00 cop[tikv] table:t keep order:false`)) + tk.MustExec(`drop table t`) + + tk.MustExec(`CREATE TABLE t (a int unsigned primary key, b int) PARTITION BY hash(a) PARTITIONS 3`) + tk.MustExec(`INSERT INTO t VALUES (1,1),(2,2),(3,3),(4,4),(5,5),(6,6)`) + tk.MustExec(`analyze table t`) + tk.MustQuery(`SELECT * FROM t WHERE a = 3`).Check(testkit.Rows("3 3")) + tk.MustQuery(`EXPLAIN FORMAT = 'brief' SELECT * FROM t WHERE a = 3`).Check(testkit.Rows("Point_Get 1.00 root table:t, partition:p0 handle:3")) +} + +func TestPruningOverflow(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec("CREATE TABLE t (a int NOT NULL, b bigint NOT NULL,PRIMARY KEY (a,b)) PARTITION BY HASH ((a*b))PARTITIONS 13") + tk.MustExec(`insert into t values(0, 3522101843073676459)`) + tk.MustQuery(`SELECT a, b FROM t WHERE a IN (0,14158354938390,0) AND b IN (3522101843073676459,-2846203247576845955,838395691793635638)`).Check(testkit.Rows("0 3522101843073676459")) +} + +func TestPartitionCoverage(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`set tidb_partition_prune_mode = 'dynamic'`) + tk.MustExec(`create table t (id int, d date, filler varchar(255))`) + tk.MustExec(`insert into t (id, d) values (1, '2024-02-29'), (2,'2024-03-01')`) + tk.MustExec(`alter table t partition by list (YEAR(d)) (partition p0 values in (2024,2025), partition p1 values in (2023))`) + tk.MustQuery(`select id,d from t partition (p0)`).Check(testkit.Rows("1 2024-02-29", "2 2024-03-01")) + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + tk.MustQuery(`select id,d from t partition (p0)`).Check(testkit.Rows("1 2024-02-29", "2 2024-03-01")) + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + tk.MustQuery(`select id,d from t partition (p1)`).Check(testkit.Rows()) + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + tk.MustQuery(`select id,d from t partition (p1)`).Check(testkit.Rows()) + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + tk.MustExec(`update t set filler = 'updated' where id = 1`) + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + tk.MustExec(`drop table t`) + tk.MustExec(`create table t (a int, b int, primary key (a,b)) partition by hash(b) partitions 3`) + tk.MustExec(`insert into t values (1,1),(1,2),(2,1),(2,2),(1,3)`) + tk.MustExec(`analyze table t all columns`) + tk.MustExec(`set tidb_partition_prune_mode = 'static'`) + query := `select * from t where a in (1,2) and b = 1 order by a` + tk.MustQuery(`explain format='brief' ` + query).Check(testkit.Rows("Batch_Point_Get 2.00 root table:t, partition:p1, clustered index:PRIMARY(a, b) keep order:true, desc:false")) + tk.MustQuery(query).Check(testkit.Rows("1 1", "2 1")) + tk.MustExec(`set tidb_partition_prune_mode = 'dynamic'`) + tk.MustQuery(`explain format='brief' ` + query).Check(testkit.Rows(""+ + "TableReader 2.00 root partition:p1 data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tikv] table:t range:[1 1,1 1], [2 1,2 1], keep order:true")) + tk.MustQuery(query).Check(testkit.Rows("1 1", "2 1")) + + query = `select * from t where a = 1 and b in (1,2)` + tk.MustExec(`set tidb_partition_prune_mode = 'static'`) + tk.MustQuery(`explain format='brief' ` + query).Check(testkit.Rows(""+ + "PartitionUnion 2.00 root ", + "├─Batch_Point_Get 2.00 root table:t, partition:p1, clustered index:PRIMARY(a, b) keep order:false, desc:false", + "└─Batch_Point_Get 2.00 root table:t, partition:p2, clustered index:PRIMARY(a, b) keep order:false, desc:false")) + + tk.MustQuery(query).Sort().Check(testkit.Rows("1 1", "1 2")) + tk.MustExec(`set tidb_partition_prune_mode = 'dynamic'`) + tk.MustQuery(`explain format='brief' ` + query).Check(testkit.Rows(""+ + "TableReader 3.00 root partition:p1,p2 data:TableRangeScan", + "└─TableRangeScan 3.00 cop[tikv] table:t range:[1 1,1 1], [1 2,1 2], keep order:false")) + tk.MustQuery(query).Sort().Check(testkit.Rows("1 1", "1 2")) + tk.MustExec(`drop table t`) + + tk.MustExec(`create table t (a int) partition by range (a) (partition p values less than (10))`) + tk.MustExec(`insert into t values (1)`) + tk.MustQuery(`explain format='brief' select * from t where a = 10`).Check(testkit.Rows(""+ + "TableReader 10.00 root partition:dual data:Selection", + "└─Selection 10.00 cop[tikv] eq(test.t.a, 10)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo")) + tk.MustExec(`analyze table t all columns`) + tk.MustQuery(`explain format='brief' select * from t where a = 10`).Check(testkit.Rows(""+ + `TableReader 1.00 root partition:dual data:Selection`, + `└─Selection 1.00 cop[tikv] eq(test.t.a, 10)`, + ` └─TableFullScan 1.00 cop[tikv] table:t keep order:false`)) + tk.MustQuery(`select * from t where a = 10`).Check(testkit.Rows()) + + tk.MustExec(`drop table t`) + tk.MustExec(`set @p=1,@q=2,@u=3;`) + tk.MustExec(`create table t(a int, b int, primary key(a)) partition by hash(a) partitions 2`) + tk.MustExec(`insert into t values(1,0),(2,0),(3,0),(4,0)`) + tk.MustQuery(`explain format = 'brief' select * from t where ((a >= 3 and a <= 1) or a = 2) and 1 = 1`).Check(testkit.Rows("Point_Get 1.00 root table:t, partition:p0 handle:2")) + tk.MustQuery(`select * from t where ((a >= 3 and a <= 1) or a = 2) and 1 = 1`).Sort().Check(testkit.Rows("2 0")) + tk.MustExec(`prepare stmt from 'select * from t where ((a >= ? and a <= ?) or a = 2) and 1 = 1'`) + tk.MustQuery(`execute stmt using @p,@p`).Sort().Check(testkit.Rows("1 0", "2 0")) + tk.MustQuery(`execute stmt using @q,@q`).Sort().Check(testkit.Rows("2 0")) + tk.MustQuery(`execute stmt using @p,@u`).Sort().Check(testkit.Rows("1 0", "2 0", "3 0")) + tk.MustQuery(`execute stmt using @u,@p`).Sort().Check(testkit.Rows("2 0")) + + tk.MustExec(`create table t19141 (c_int int, primary key (c_int)) partition by hash ( c_int ) partitions 4`) + tk.MustExec(`insert into t19141 values (1), (2), (3), (4)`) + tk.MustQuery(`explain format = 'brief' select * from t19141 partition (p0)`).Check(testkit.Rows(""+ + "TableReader 10000.00 root partition:p0 data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t19141 keep order:false, stats:pseudo")) + tk.MustQuery(`select * from t19141 partition (p0)`).Sort().Check(testkit.Rows("4")) + tk.MustQuery(`select * from t19141 partition (p0) where c_int = 1`).Sort().Check(testkit.Rows()) + tk.MustExec(`update t19141 partition (p0) set c_int = -c_int where c_int = 1`) + tk.MustQuery(`select * from t19141 order by c_int`).Sort().Check(testkit.Rows("1", "2", "3", "4")) + tk.MustQuery(`select * from t19141 partition (p0, p2) where c_int in (1,2,3)`).Sort().Check(testkit.Rows("2")) + tk.MustExec(`update t19141 partition (p1) set c_int = -c_int where c_int in (2,3)`) + tk.MustQuery(`select * from t19141 order by c_int`).Sort().Check(testkit.Rows("1", "2", "3", "4")) + tk.MustExec(`delete from t19141 partition (p0) where c_int in (2,3)`) + tk.MustQuery(`select * from t19141 order by c_int`).Sort().Check(testkit.Rows("1", "2", "3", "4")) +} + +// Issue TiDB #51090. +func TestAlterTablePartitionRollback(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk3 := testkit.NewTestKit(t, store) + tk4 := testkit.NewTestKit(t, store) + tk5 := testkit.NewTestKit(t, store) + tk.MustExec(`use test;`) + tk2.MustExec(`use test;`) + tk3.MustExec(`use test;`) + tk4.MustExec(`use test;`) + tk5.MustExec(`use test;`) + tk.MustExec(`create table t(a int);`) + tk.MustExec(`insert into t values(1), (2), (3);`) + + alterChan := make(chan error) + alterPartition := func() { + err := tk4.ExecToErr(`alter table t partition by hash(a) partitions 3;`) + alterChan <- err + } + waitFor := func(s string) { + for { + select { + case alterErr := <-alterChan: + require.Fail(t, "Alter completed unexpectedly", "With error %v", alterErr) + default: + // Alter still running + } + res := tk5.MustQuery(`admin show ddl jobs where db_name = 'test' and table_name = 't' and job_type = 'alter table partition by'`).Rows() + if len(res) > 0 && res[0][4] == s { + logutil.BgLogger().Info("Got state", zap.String("State", s)) + break + } + gotime.Sleep(10 * gotime.Millisecond) + } + dom := domain.GetDomain(tk5.Session()) + // Make sure the table schema is the new schema. + require.NoError(t, dom.Reload()) + } + + testFunc := func(states []string) { + for i, s := range states { + if i%2 == 0 { + tk2.MustExec(`begin;`) + tk2.MustExec(`select 1 from t;`) + if i > 0 { + tk3.MustExec(`commit;`) + } + } else { + tk3.MustExec(`begin;`) + tk3.MustExec(`select 1 from t;`) + tk2.MustExec(`commit;`) + } + if i == 0 { + go alterPartition() + } + waitFor(s) + if i == len(states)-1 { + break + } + } + res := tk.MustQuery(`admin show ddl jobs where table_name = 't' and job_type = 'alter table partition by'`).Rows() + tk.MustExec(fmt.Sprintf("admin cancel ddl jobs %v", res[0][0])) + tk2.MustExec(`commit;`) + tk3.MustExec(`commit;`) + require.ErrorContains(t, <-alterChan, "[ddl:8214]Cancelled DDL job") + tk.MustQuery(`show create table t;`).Check(testkit.Rows( + "t CREATE TABLE `t` (\n" + + " `a` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tk.MustQuery(`select a from t order by a;`).Check(testkit.Rows("1", "2", "3")) + } + + states := []string{"delete only", "write only", "write reorganization", "delete reorganization"} + for i := range states { + testFunc(states[:i+1]) + } +} diff --git a/planner/core/testdata/analyze_suite_out.json b/planner/core/testdata/analyze_suite_out.json index ea1e33b3ca0b8..f3b65f38d4f9b 100644 --- a/planner/core/testdata/analyze_suite_out.json +++ b/planner/core/testdata/analyze_suite_out.json @@ -364,13 +364,13 @@ "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t keep order:false" ], [ - "TableReader 0.00 root data:Selection", - "└─Selection 0.00 cop[tikv] eq(test.t.b, 1)", + "TableReader 1.00 root data:Selection", + "└─Selection 1.00 cop[tikv] eq(test.t.b, 1)", " └─TableFullScan 2.00 cop[tikv] table:t keep order:false" ], [ - "TableReader 0.00 root data:Selection", - "└─Selection 0.00 cop[tikv] lt(test.t.b, 1)", + "TableReader 1.00 root data:Selection", + "└─Selection 1.00 cop[tikv] lt(test.t.b, 1)", " └─TableFullScan 2.00 cop[tikv] table:t keep order:false" ] ] diff --git a/planner/core/testdata/index_merge_suite_out.json b/planner/core/testdata/index_merge_suite_out.json index 3d67e5e372251..1b9ade46ff5eb 100644 --- a/planner/core/testdata/index_merge_suite_out.json +++ b/planner/core/testdata/index_merge_suite_out.json @@ -251,7 +251,7 @@ { "SQL": "select * from vh", "Plan": [ - "PartitionUnion 0.50 root ", + "PartitionUnion 1.50 root ", "├─IndexMerge 0.50 root type: intersection", "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", @@ -275,7 +275,7 @@ { "SQL": "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", "Plan": [ - "PartitionUnion 0.50 root ", + "PartitionUnion 1.50 root ", "├─IndexMerge 0.50 root type: intersection", "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", @@ -299,7 +299,7 @@ { "SQL": "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", "Plan": [ - "PartitionUnion 0.50 root ", + "PartitionUnion 1.50 root ", "├─IndexMerge 0.50 root type: intersection", "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", @@ -323,7 +323,7 @@ { "SQL": "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", "Plan": [ - "PartitionUnion 0.50 root ", + "PartitionUnion 1.50 root ", "├─IndexMerge 0.50 root type: intersection", "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", @@ -347,7 +347,7 @@ { "SQL": "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", "Plan": [ - "PartitionUnion 0.50 root ", + "PartitionUnion 1.50 root ", "├─IndexMerge 0.50 root type: intersection", "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", diff --git a/planner/core/testdata/plan_normalized_suite_out.json b/planner/core/testdata/plan_normalized_suite_out.json index 70de129af4de6..46737869dd90b 100644 --- a/planner/core/testdata/plan_normalized_suite_out.json +++ b/planner/core/testdata/plan_normalized_suite_out.json @@ -339,5 +339,143 @@ ] } ] +<<<<<<< HEAD:planner/core/testdata/plan_normalized_suite_out.json +======= + }, + { + "Name": "TestTiFlashLateMaterialization", + "Cases": [ + { + "SQL": "explain select * from t1;", + "Plan": [ + " TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], keep order:false" + ] + }, + { + "SQL": "explain select * from t1 where a<1;", + "Plan": [ + " TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:lt(test.t1.a, ?), keep order:false" + ] + }, + { + "SQL": "explain select * from t1 where a>1", + "Plan": [ + " TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.a, ?), keep order:false" + ] + }, + { + "SQL": "explain select * from t1 where a=1", + "Plan": [ + " TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:eq(test.t1.a, ?), keep order:false" + ] + }, + { + "SQL": "explain select * from t1 where a in (1,2,3)", + "Plan": [ + " TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─Selection cop[tiflash] in(test.t1.a, ?, ?, ?)", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:empty, keep order:false" + ] + }, + { + "SQL": "explain select * from t1 where b=1", + "Plan": [ + " TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:eq(test.t1.b, ?), keep order:false" + ] + }, + { + "SQL": "explain select * from t1 where a!=1 order by c limit 1", + "Plan": [ + " TopN root test.t1.c", + " └─TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─TopN cop[tiflash] test.t1.c", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:ne(test.t1.a, ?), keep order:false" + ] + }, + { + "SQL": "explain select a from t1 where a>1", + "Plan": [ + " TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─Selection cop[tiflash] gt(test.t1.a, ?)", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:empty, keep order:false" + ] + }, + { + "SQL": "explain select a from t1 where a>1 and b>1", + "Plan": [ + " TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─Projection cop[tiflash] test.t1.a", + " └─Selection cop[tiflash] gt(test.t1.b, ?)", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.a, ?), keep order:false" + ] + }, + { + "SQL": "explain select * from t1 where a>1 and b>1 and c>1", + "Plan": [ + " TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─Selection cop[tiflash] gt(test.t1.c, ?)", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.a, ?), gt(test.t1.b, ?), keep order:false" + ] + }, + { + "SQL": "explain select * from t1 where a<1 or b<2", + "Plan": [ + " TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:or(lt(test.t1.a, ?), lt(test.t1.b, ?)), keep order:false" + ] + }, + { + "SQL": "explain select * from t1 where (a<1 or b<2) and (a>3 and b>3)", + "Plan": [ + " TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─Selection cop[tiflash] gt(test.t1.b, ?), or(lt(test.t1.a, ?), lt(test.t1.b, ?))", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.a, ?), keep order:false" + ] + }, + { + "SQL": "explain select * from t1 where (a<1 or b<2) and (a>3 and b>3) and c>1", + "Plan": [ + " TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─Selection cop[tiflash] gt(test.t1.b, ?), gt(test.t1.c, ?), or(lt(test.t1.a, ?), lt(test.t1.b, ?))", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.a, ?), keep order:false" + ] + }, + { + "SQL": "explain select * from t1 where (a>2 or b<2) and (a>3 and b>3) and c>2", + "Plan": [ + " TableReader root ", + " └─ExchangeSender cop[tiflash] ", + " └─Selection cop[tiflash] gt(test.t1.a, ?), gt(test.t1.c, ?), or(gt(test.t1.a, ?), lt(test.t1.b, ?))", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.b, ?), keep order:false" + ] + }, + { + "SQL": "explain select count(a), max(t) from t1 where a>1", + "Plan": [ + " HashAgg root funcs:count(test.t1.a)->?, funcs:max(test.t1.t)->?", + " └─TableReader root ", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.a, ?), keep order:false" + ] + } + ] +>>>>>>> f2c278ddc6b (Planner: Do not allow cardinality to go below 1 (#55242)):pkg/planner/core/casetest/testdata/plan_normalized_suite_out.json } ] diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index fca2f9261c632..1ddf4b990109f 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -224,7 +224,7 @@ func TestEstimationForUnknownValues(t *testing.T) { colID := table.Meta().Columns[0].ID count, err := statsTbl.GetRowCountByColumnRanges(sctx, colID, getRange(30, 30)) require.NoError(t, err) - require.Equal(t, 0.2, count) + require.Equal(t, 1.0, count) count, err = statsTbl.GetRowCountByColumnRanges(sctx, colID, getRange(9, 30)) require.NoError(t, err) @@ -253,7 +253,7 @@ func TestEstimationForUnknownValues(t *testing.T) { colID = table.Meta().Columns[0].ID count, err = statsTbl.GetRowCountByColumnRanges(sctx, colID, getRange(1, 30)) require.NoError(t, err) - require.Equal(t, 0.0, count) + require.Equal(t, 1.0, count) testKit.MustExec("drop table t") testKit.MustExec("create table t(a int, b int, index idx(b))") @@ -266,7 +266,7 @@ func TestEstimationForUnknownValues(t *testing.T) { colID = table.Meta().Columns[0].ID count, err = statsTbl.GetRowCountByColumnRanges(sctx, colID, getRange(2, 2)) require.NoError(t, err) - require.Equal(t, 0.0, count) + require.Equal(t, 1.0, count) idxID = table.Meta().Indices[0].ID count, err = statsTbl.GetRowCountByIndexRanges(sctx, idxID, getRange(2, 2)) @@ -522,8 +522,8 @@ func TestSelectivity(t *testing.T) { }, { exprs: "a >= 1 and b > 1 and a < 2", - selectivity: 0.01783264746, - selectivityAfterIncrease: 0.01851851852, + selectivity: 0.017832647462277088, + selectivityAfterIncrease: 0.018518518518518517, }, { exprs: "a >= 1 and c > 1 and a < 2", @@ -542,13 +542,13 @@ func TestSelectivity(t *testing.T) { }, { exprs: "b > 1", - selectivity: 0.96296296296, + selectivity: 0.9629629629629629, selectivityAfterIncrease: 1, }, { exprs: "a > 1 and b < 2 and c > 3 and d < 4 and e > 5", - selectivity: 0, - selectivityAfterIncrease: 0, + selectivity: 5.870830440255832e-05, + selectivityAfterIncrease: 1.51329827770157e-05, }, { exprs: longExpr, @@ -1157,3 +1157,475 @@ func TestGlobalStatsOutOfRangeEstimationAfterDelete(t *testing.T) { testKit.MustQuery(input[i]).Check(testkit.Rows(output[i].Result...)) } } +<<<<<<< HEAD:statistics/selectivity_test.go +======= + +func generateMapsForMockStatsTbl(statsTbl *statistics.Table) { + idx2Columns := make(map[int64][]int64) + colID2IdxIDs := make(map[int64][]int64) + statsTbl.ForEachIndexImmutable(func(_ int64, idxHist *statistics.Index) bool { + ids := make([]int64, 0, len(idxHist.Info.Columns)) + for _, idxCol := range idxHist.Info.Columns { + ids = append(ids, int64(idxCol.Offset)) + } + colID2IdxIDs[ids[0]] = append(colID2IdxIDs[ids[0]], idxHist.ID) + idx2Columns[idxHist.ID] = ids + return false + }) + for _, idxIDs := range colID2IdxIDs { + slices.Sort(idxIDs) + } + statsTbl.Idx2ColUniqueIDs = idx2Columns + statsTbl.ColUniqueID2IdxIDs = colID2IdxIDs +} + +func TestIssue39593(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(a int, b int, index idx(a, b))") + is := dom.InfoSchema() + tb, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo := tb.Meta() + + // mock the statistics.Table + statsTbl := mockStatsTable(tblInfo, 540) + colValues, err := generateIntDatum(1, 54) + require.NoError(t, err) + for i := 1; i <= 2; i++ { + statsTbl.SetCol(int64(i), &statistics.Column{ + Histogram: *mockStatsHistogram(int64(i), colValues, 10, types.NewFieldType(mysql.TypeLonglong)), + Info: tblInfo.Columns[i-1], + StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), + StatsVer: 2, + }) + } + idxValues, err := generateIntDatum(2, 3) + require.NoError(t, err) + tp := types.NewFieldType(mysql.TypeBlob) + statsTbl.SetIdx(1, &statistics.Index{ + Histogram: *mockStatsHistogram(1, idxValues, 60, tp), + Info: tblInfo.Indices[0], + StatsVer: 2, + }) + generateMapsForMockStatsTbl(statsTbl) + + sctx := testKit.Session() + idxID := tblInfo.Indices[0].ID + vals := []int64{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20} + count, err := cardinality.GetRowCountByIndexRanges(sctx.GetPlanCtx(), &statsTbl.HistColl, idxID, getRanges(vals, vals)) + require.NoError(t, err) + // estimated row count without any changes + require.Equal(t, float64(360), count) + statsTbl.RealtimeCount *= 10 + count, err = cardinality.GetRowCountByIndexRanges(sctx.GetPlanCtx(), &statsTbl.HistColl, idxID, getRanges(vals, vals)) + require.NoError(t, err) + // estimated row count after mock modify on the table + require.Equal(t, float64(3600), count) +} + +func TestIndexJoinInnerRowCountUpperBound(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(a int, b int, index idx(b))") + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + is := dom.InfoSchema() + tb, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo := tb.Meta() + + // Mock the stats: + // The two columns are the same. + // From 0 to 499, each value has 1000 rows. Therefore, NDV is 500 and total row count is 500000. + mockStatsTbl := mockStatsTable(tblInfo, 500000) + colValues, err := generateIntDatum(1, 500) + require.NoError(t, err) + for i := 1; i <= 2; i++ { + mockStatsTbl.SetCol(int64(i), &statistics.Column{ + Histogram: *mockStatsHistogram(int64(i), colValues, 1000, types.NewFieldType(mysql.TypeLonglong)), + Info: tblInfo.Columns[i-1], + StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), + StatsVer: 2, + }) + } + idxValues := make([]types.Datum, 0, len(colValues)) + sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) + for _, colV := range colValues { + b, err := codec.EncodeKey(sc.TimeZone(), nil, colV) + require.NoError(t, err) + idxValues = append(idxValues, types.NewBytesDatum(b)) + } + mockStatsTbl.SetIdx(1, &statistics.Index{ + Histogram: *mockStatsHistogram(1, idxValues, 1000, types.NewFieldType(mysql.TypeBlob)), + Info: tblInfo.Indices[0], + StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), + StatsVer: 2, + }) + generateMapsForMockStatsTbl(mockStatsTbl) + stat := h.GetTableStats(tblInfo) + stat.HistColl = mockStatsTbl.HistColl + + var ( + input []string + output []struct { + Query string + Result []string + } + ) + + suiteData := cardinality.GetCardinalitySuiteData() + suiteData.LoadTestCases(t, &input, &output) + for i := 0; i < len(input); i++ { + testdata.OnRecord(func() { + output[i].Query = input[i] + }) + if !strings.HasPrefix(input[i], "explain") { + testKit.MustExec(input[i]) + continue + } + testdata.OnRecord(func() { + output[i].Result = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + }) + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i].Result...)) + } +} + +func TestOrderingIdxSelectivityThreshold(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) + + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(a int primary key , b int, c int, index ib(b), index ic(c))") + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + is := dom.InfoSchema() + tb, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo := tb.Meta() + + // Mock the stats: + // total row count 100000 + // column a: PK, from 0 to 100000, NDV 100000 + // column b, c: from 0 to 10000, each value has 10 rows, NDV 10000 + // indexes are created on (b), (c) respectively + mockStatsTbl := mockStatsTable(tblInfo, 100000) + pkColValues, err := generateIntDatum(1, 100000) + require.NoError(t, err) + mockStatsTbl.SetCol(1, &statistics.Column{ + Histogram: *mockStatsHistogram(1, pkColValues, 1, types.NewFieldType(mysql.TypeLonglong)), + Info: tblInfo.Columns[0], + StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), + StatsVer: 2, + }) + colValues, err := generateIntDatum(1, 10000) + require.NoError(t, err) + idxValues := make([]types.Datum, 0) + for _, val := range colValues { + b, err := codec.EncodeKey(sc.TimeZone(), nil, val) + require.NoError(t, err) + idxValues = append(idxValues, types.NewBytesDatum(b)) + } + + for i := 2; i <= 3; i++ { + mockStatsTbl.SetCol(int64(i), &statistics.Column{ + Histogram: *mockStatsHistogram(int64(i), colValues, 10, types.NewFieldType(mysql.TypeLonglong)), + Info: tblInfo.Columns[i-1], + StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), + StatsVer: 2, + }) + } + for i := 1; i <= 2; i++ { + mockStatsTbl.SetIdx(int64(i), &statistics.Index{ + Histogram: *mockStatsHistogram(int64(i), idxValues, 10, types.NewFieldType(mysql.TypeBlob)), + Info: tblInfo.Indices[i-1], + StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), + StatsVer: 2, + }) + } + generateMapsForMockStatsTbl(mockStatsTbl) + stat := h.GetTableStats(tblInfo) + stat.HistColl = mockStatsTbl.HistColl + + var ( + input []string + output []struct { + Query string + Result []string + } + ) + + integrationSuiteData := cardinality.GetCardinalitySuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i := 0; i < len(input); i++ { + testdata.OnRecord(func() { + output[i].Query = input[i] + }) + if !strings.HasPrefix(input[i], "explain") { + testKit.MustExec(input[i]) + continue + } + testdata.OnRecord(func() { + output[i].Result = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + }) + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i].Result...)) + } +} + +func TestOrderingIdxSelectivityRatio(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) + + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(a int primary key, b int, c int, index ib(b), index ic(c))") + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + is := dom.InfoSchema() + tb, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo := tb.Meta() + + // Mock the stats: + // total row count 1000 + // column b, c: from 1 to 1000, NDV 1000 + // indexes are created on (b), (c) respectively + mockStatsTbl := mockStatsTable(tblInfo, 1000) + pkColValues, err := generateIntDatum(1, 1000) + require.NoError(t, err) + mockStatsTbl.SetCol(1, &statistics.Column{ + Histogram: *mockStatsHistogram(1, pkColValues, 1, types.NewFieldType(mysql.TypeLonglong)), + Info: tblInfo.Columns[0], + StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), + StatsVer: 2, + }) + colValues, err := generateIntDatum(1, 1000) + require.NoError(t, err) + idxValues := make([]types.Datum, 0) + for _, val := range colValues { + b, err := codec.EncodeKey(sc.TimeZone(), nil, val) + require.NoError(t, err) + idxValues = append(idxValues, types.NewBytesDatum(b)) + } + + for i := 2; i <= 3; i++ { + mockStatsTbl.SetCol(int64(i), &statistics.Column{ + Histogram: *mockStatsHistogram(int64(i), colValues, 1, types.NewFieldType(mysql.TypeLonglong)), + Info: tblInfo.Columns[i-1], + StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), + StatsVer: 2, + }) + } + for i := 1; i <= 2; i++ { + mockStatsTbl.SetIdx(int64(i), &statistics.Index{ + Histogram: *mockStatsHistogram(int64(i), idxValues, 1, types.NewFieldType(mysql.TypeBlob)), + Info: tblInfo.Indices[i-1], + StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), + StatsVer: 2, + }) + } + generateMapsForMockStatsTbl(mockStatsTbl) + stat := h.GetTableStats(tblInfo) + stat.HistColl = mockStatsTbl.HistColl + + var ( + input []string + output []struct { + Query string + Result []string + } + ) + + integrationSuiteData := cardinality.GetCardinalitySuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i := 0; i < len(input); i++ { + testdata.OnRecord(func() { + output[i].Query = input[i] + }) + if !strings.HasPrefix(input[i], "explain") { + testKit.MustExec(input[i]) + continue + } + testdata.OnRecord(func() { + output[i].Result = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + }) + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i].Result...)) + } +} + +func TestCrossValidationSelectivity(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("set @@tidb_analyze_version = 1") + tk.MustExec("create table t (a int, b int, c int, primary key (a, b) clustered)") + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + tk.MustExec("insert into t values (1,2,3), (1,4,5)") + require.NoError(t, h.DumpStatsDeltaToKV(true)) + tk.MustExec("analyze table t") + tk.MustQuery("explain format = 'brief' select * from t where a = 1 and b > 0 and b < 1000 and c > 1000").Check(testkit.Rows( + "TableReader 1.00 root data:Selection", + "└─Selection 1.00 cop[tikv] gt(test.t.c, 1000)", + " └─TableRangeScan 2.00 cop[tikv] table:t range:(1 0,1 1000), keep order:false")) +} + +func TestIgnoreRealtimeStats(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(a int, b int, index ib(b))") + h := dom.StatsHandle() + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + + // 1. Insert 11 rows of data without ANALYZE. + testKit.MustExec("insert into t values(1,1),(1,2),(1,3),(1,4),(1,5),(2,1),(2,2),(2,3),(2,4),(2,5),(3,1)") + require.Nil(t, h.DumpStatsDeltaToKV(true)) + require.Nil(t, h.Update(context.Background(), dom.InfoSchema())) + + // 1-1. use real-time stats. + // From the real-time stats, we are able to know the total count is 11. + testKit.MustExec("set @@tidb_opt_objective = 'moderate'") + testKit.MustQuery("explain select * from t where a = 1 and b > 2").Check(testkit.Rows( + "TableReader_7 0.00 root data:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.t.a, 1), gt(test.t.b, 2)", + " └─TableFullScan_5 11.00 cop[tikv] table:t keep order:false, stats:pseudo", + )) + + // 1-2. ignore real-time stats. + // Use pseudo stats table. The total row count is 10000. + testKit.MustExec("set @@tidb_opt_objective = 'determinate'") + testKit.MustQuery("explain select * from t where a = 1 and b > 2").Check(testkit.Rows( + "TableReader_7 3.33 root data:Selection_6", + "└─Selection_6 3.33 cop[tikv] eq(test.t.a, 1), gt(test.t.b, 2)", + " └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + )) + + // 2. After ANALYZE. + testKit.MustExec("analyze table t all columns with 1 samplerate") + require.Nil(t, h.Update(context.Background(), dom.InfoSchema())) + + // The execution plans are the same no matter we ignore the real-time stats or not. + analyzedPlan := []string{ + "TableReader_7 2.73 root data:Selection_6", + "└─Selection_6 2.73 cop[tikv] eq(test.t.a, 1), gt(test.t.b, 2)", + " └─TableFullScan_5 11.00 cop[tikv] table:t keep order:false", + } + testKit.MustExec("set @@tidb_opt_objective = 'moderate'") + testKit.MustQuery("explain select * from t where a = 1 and b > 2").Check(testkit.Rows(analyzedPlan...)) + testKit.MustExec("set @@tidb_opt_objective = 'determinate'") + testKit.MustQuery("explain select * from t where a = 1 and b > 2").Check(testkit.Rows(analyzedPlan...)) + + // 3. Insert another 4 rows of data. + testKit.MustExec("insert into t values(3,2),(3,3),(3,4),(3,5)") + require.Nil(t, h.DumpStatsDeltaToKV(true)) + require.Nil(t, h.Update(context.Background(), dom.InfoSchema())) + + // 3-1. use real-time stats. + // From the real-time stats, we are able to know the total count is 15. + // Selectivity is not changed: 15 * (2.73 / 11) = 3.72 + testKit.MustExec("set @@tidb_opt_objective = 'moderate'") + testKit.MustQuery("explain select * from t where a = 1 and b > 2").Check(testkit.Rows( + "TableReader_7 3.72 root data:Selection_6", + "└─Selection_6 3.72 cop[tikv] eq(test.t.a, 1), gt(test.t.b, 2)", + " └─TableFullScan_5 15.00 cop[tikv] table:t keep order:false", + )) + + // 3-2. ignore real-time stats. + // The execution plan is the same as case 2. + testKit.MustExec("set @@tidb_opt_objective = 'determinate'") + testKit.MustQuery("explain select * from t where a = 1 and b > 2").Check(testkit.Rows(analyzedPlan...)) +} + +func TestSubsetIdxCardinality(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(a int, b int, c int, index iabc(a, b, c))") + // Insert enough rows with differing cardinalities to test subset vs full index cardinality estimate. + // Result of a 2 column match should produce more rows than 3 column match. + testKit.MustExec("insert into t values (1, 1, 1), (1, 1, 1), (2, 1, 1), (2, 1, 1), (3, 1, 1), (3, 1, 1), (4, 1, 1), (4, 1, 1), (5, 1, 1), (5, 1, 1)") + testKit.MustExec("insert into t select a + 5, a, a from t") + for i := 1; i < 3; i++ { + testKit.MustExec(fmt.Sprintf("insert into t select a + 10 + %v, b + 1, c from t", i)) + } + for j := 0; j < 3; j++ { + testKit.MustExec("insert into t select a, b, c from t") + } + testKit.MustExec("insert into t select a, b + 10, c from t") + require.NoError(t, h.DumpStatsDeltaToKV(true)) + testKit.MustExec(`analyze table t`) + + var ( + input []string + output []struct { + Query string + Result []string + } + ) + integrationSuiteData := cardinality.GetCardinalitySuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i := 0; i < len(input); i++ { + testdata.OnRecord(func() { + output[i].Query = input[i] + }) + if !strings.HasPrefix(input[i], "explain") { + testKit.MustExec(input[i]) + continue + } + testdata.OnRecord(func() { + output[i].Result = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + }) + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i].Result...)) + } +} + +func TestBuiltinInEstWithoutStats(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + + tk.MustExec("use test") + tk.MustExec("create table t(a int)") + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + tk.MustExec("insert into t values(1), (2), (3), (4), (5), (6), (7), (8), (9), (10)") + require.NoError(t, h.DumpStatsDeltaToKV(true)) + is := dom.InfoSchema() + require.NoError(t, h.Update(context.Background(), is)) + + tk.MustQuery("explain format='brief' select * from t where a in (1, 2, 3, 4, 5, 6, 7, 8)").Check(testkit.Rows( + "TableReader 0.08 root data:Selection", + "└─Selection 0.08 cop[tikv] in(test.t.a, 1, 2, 3, 4, 5, 6, 7, 8)", + " └─TableFullScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo", + )) + + h.Clear() + require.NoError(t, h.InitStatsLite(context.Background(), is)) + tk.MustQuery("explain format='brief' select * from t where a in (1, 2, 3, 4, 5, 6, 7, 8)").Check(testkit.Rows( + "TableReader 0.08 root data:Selection", + "└─Selection 0.08 cop[tikv] in(test.t.a, 1, 2, 3, 4, 5, 6, 7, 8)", + " └─TableFullScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo", + )) + h.Clear() + require.NoError(t, h.InitStats(context.Background(), is)) + tk.MustQuery("explain format='brief' select * from t where a in (1, 2, 3, 4, 5, 6, 7, 8)").Check(testkit.Rows( + "TableReader 8.00 root data:Selection", + "└─Selection 8.00 cop[tikv] in(test.t.a, 1, 2, 3, 4, 5, 6, 7, 8)", + " └─TableFullScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo", + )) +} +>>>>>>> f2c278ddc6b (Planner: Do not allow cardinality to go below 1 (#55242)):pkg/planner/cardinality/selectivity_test.go diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index ee6d871580af3..db07491f9c202 100644 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -480,7 +480,7 @@ func SubTestIndexRanges() func(*testing.T) { ran[0].HighVal[0] = types.NewIntDatum(1000) count, err = tbl.GetRowCountByIndexRanges(ctx, 0, ran) require.NoError(t, err) - require.Equal(t, 0, int(count)) + require.Equal(t, 1, int(count)) } } diff --git a/tests/integrationtest/r/executor/partition/partition_boundaries.result b/tests/integrationtest/r/executor/partition/partition_boundaries.result new file mode 100644 index 0000000000000..5148ee5e34696 --- /dev/null +++ b/tests/integrationtest/r/executor/partition/partition_boundaries.result @@ -0,0 +1,5260 @@ +SET @@tidb_partition_prune_mode = 'dynamic'; +DROP TABLE IF EXISTS t; +CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( +PARTITION p0 VALUES LESS THAN (1000000), +PARTITION p1 VALUES LESS THAN (2000000), +PARTITION p2 VALUES LESS THAN (3000000)); +INSERT INTO t VALUES (999998, '999998 Filler ...'), (999999, '999999 Filler ...'), (1000000, '1000000 Filler ...'), (1000001, '1000001 Filler ...'), (1000002, '1000002 Filler ...'); +INSERT INTO t VALUES (1999998, '1999998 Filler ...'), (1999999, '1999999 Filler ...'), (2000000, '2000000 Filler ...'), (2000001, '2000001 Filler ...'), (2000002, '2000002 Filler ...'); +INSERT INTO t VALUES (2999998, '2999998 Filler ...'), (2999999, '2999999 Filler ...'); +INSERT INTO t VALUES (-2147483648, 'MIN_INT filler...'), (0, '0 Filler...'); +ANALYZE TABLE t all columns; +explain format='brief' SELECT * FROM t WHERE a = -2147483648; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, -2147483648) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a = -2147483648; +a b +-2147483648 MIN_INT filler... +explain format='brief' SELECT * FROM t WHERE a IN (-2147483648); +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, -2147483648) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (-2147483648); +a b +-2147483648 MIN_INT filler... +explain format='brief' SELECT * FROM t WHERE a = 0; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a = 0; +a b +0 0 Filler... +explain format='brief' SELECT * FROM t WHERE a IN (0); +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (0); +a b +0 0 Filler... +explain format='brief' SELECT * FROM t WHERE a = 999998; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a = 999998; +a b +999998 999998 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (999998); +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (999998); +a b +999998 999998 Filler ... +explain format='brief' SELECT * FROM t WHERE a = 999999; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a = 999999; +a b +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (999999); +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (999999); +a b +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a = 1000000; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a = 1000000; +a b +1000000 1000000 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (1000000); +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (1000000); +a b +1000000 1000000 Filler ... +explain format='brief' SELECT * FROM t WHERE a = 1000001; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a = 1000001; +a b +1000001 1000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (1000001); +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (1000001); +a b +1000001 1000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a = 1000002; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a = 1000002; +a b +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (1000002); +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (1000002); +a b +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a = 3000000; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a = 3000000; +a b +explain format='brief' SELECT * FROM t WHERE a IN (3000000); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (3000000); +a b +explain format='brief' SELECT * FROM t WHERE a = 3000001; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 3000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a = 3000001; +a b +explain format='brief' SELECT * FROM t WHERE a IN (3000001); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_boundaries.t.a, 3000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (3000001); +a b +explain format='brief' SELECT * FROM t WHERE a IN (-2147483648, -2147483647); +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] in(executor__partition__partition_boundaries.t.a, -2147483648, -2147483647) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (-2147483648, -2147483647); +a b +-2147483648 MIN_INT filler... +explain format='brief' SELECT * FROM t WHERE a IN (-2147483647, -2147483646); +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] in(executor__partition__partition_boundaries.t.a, -2147483647, -2147483646) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (-2147483647, -2147483646); +a b +explain format='brief' SELECT * FROM t WHERE a IN (999997, 999998, 999999); +id estRows task access object operator info +TableReader 2.00 root partition:p0 data:Selection +└─Selection 2.00 cop[tikv] in(executor__partition__partition_boundaries.t.a, 999997, 999998, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (999997, 999998, 999999); +a b +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (999998, 999999, 1000000); +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1 data:Selection +└─Selection 3.00 cop[tikv] in(executor__partition__partition_boundaries.t.a, 999998, 999999, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (999998, 999999, 1000000); +a b +1000000 1000000 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (999999, 1000000, 1000001); +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1 data:Selection +└─Selection 3.00 cop[tikv] in(executor__partition__partition_boundaries.t.a, 999999, 1000000, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (999999, 1000000, 1000001); +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (1000000, 1000001, 1000002); +id estRows task access object operator info +TableReader 3.00 root partition:p1 data:Selection +└─Selection 3.00 cop[tikv] in(executor__partition__partition_boundaries.t.a, 1000000, 1000001, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (1000000, 1000001, 1000002); +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (1999997, 1999998, 1999999); +id estRows task access object operator info +TableReader 2.00 root partition:p1 data:Selection +└─Selection 2.00 cop[tikv] in(executor__partition__partition_boundaries.t.a, 1999997, 1999998, 1999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (1999997, 1999998, 1999999); +a b +1999998 1999998 Filler ... +1999999 1999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (1999998, 1999999, 2000000); +id estRows task access object operator info +TableReader 3.00 root partition:p1,p2 data:Selection +└─Selection 3.00 cop[tikv] in(executor__partition__partition_boundaries.t.a, 1999998, 1999999, 2000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (1999998, 1999999, 2000000); +a b +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (1999999, 2000000, 2000001); +id estRows task access object operator info +TableReader 3.00 root partition:p1,p2 data:Selection +└─Selection 3.00 cop[tikv] in(executor__partition__partition_boundaries.t.a, 1999999, 2000000, 2000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (1999999, 2000000, 2000001); +a b +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (2000000, 2000001, 2000002); +id estRows task access object operator info +TableReader 3.00 root partition:p2 data:Selection +└─Selection 3.00 cop[tikv] in(executor__partition__partition_boundaries.t.a, 2000000, 2000001, 2000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (2000000, 2000001, 2000002); +a b +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (2999997, 2999998, 2999999); +id estRows task access object operator info +TableReader 2.00 root partition:p2 data:Selection +└─Selection 2.00 cop[tikv] in(executor__partition__partition_boundaries.t.a, 2999997, 2999998, 2999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (2999997, 2999998, 2999999); +a b +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (2999998, 2999999, 3000000); +id estRows task access object operator info +TableReader 2.00 root partition:p2 data:Selection +└─Selection 2.00 cop[tikv] in(executor__partition__partition_boundaries.t.a, 2999998, 2999999, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (2999998, 2999999, 3000000); +a b +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (2999999, 3000000, 3000001); +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] in(executor__partition__partition_boundaries.t.a, 2999999, 3000000, 3000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (2999999, 3000000, 3000001); +a b +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a IN (3000000, 3000001, 3000002); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] in(executor__partition__partition_boundaries.t.a, 3000000, 3000001, 3000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a IN (3000000, 3000001, 3000002); +a b +SET @@tidb_partition_prune_mode = default; +SET @@tidb_partition_prune_mode = 'dynamic'; +DROP TABLE IF EXISTS t; +CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( +PARTITION p0 VALUES LESS THAN (1), +PARTITION p1 VALUES LESS THAN (2), +PARTITION p2 VALUES LESS THAN (3), +PARTITION p3 VALUES LESS THAN (4), +PARTITION p4 VALUES LESS THAN (5), +PARTITION p5 VALUES LESS THAN (6), +PARTITION p6 VALUES LESS THAN (7)); +INSERT INTO t VALUES (0, '0 Filler...'); +INSERT INTO t VALUES (1, '1 Filler...'); +INSERT INTO t VALUES (2, '2 Filler...'); +INSERT INTO t VALUES (3, '3 Filler...'); +INSERT INTO t VALUES (4, '4 Filler...'); +INSERT INTO t VALUES (5, '5 Filler...'); +INSERT INTO t VALUES (6, '6 Filler...'); +ANALYZE TABLE t all columns; +explain format='brief' SELECT * FROM t WHERE a != -1; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, -1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a != -1; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 1 AND a != -1; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, -1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 1 AND a != -1; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a NOT IN (-2, -1); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] not(in(executor__partition__partition_boundaries.t.a, -2, -1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a NOT IN (-2, -1); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 0 OR a = -1; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] or(0, eq(executor__partition__partition_boundaries.t.a, -1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 0 OR a = -1; +a b +explain format='brief' SELECT * FROM t WHERE a != 0; +id estRows task access object operator info +TableReader 6.00 root partition:all data:Selection +└─Selection 6.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a != 0; +a b +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0; +id estRows task access object operator info +TableReader 6.00 root partition:all data:Selection +└─Selection 6.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, -1), ne(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0; +a b +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a NOT IN (-2, -1, 0); +id estRows task access object operator info +TableReader 6.00 root partition:all data:Selection +└─Selection 6.00 cop[tikv] not(in(executor__partition__partition_boundaries.t.a, -2, -1, 0)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a NOT IN (-2, -1, 0); +a b +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] or(0, or(eq(executor__partition__partition_boundaries.t.a, -1), eq(executor__partition__partition_boundaries.t.a, 0))) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0; +a b +0 0 Filler... +explain format='brief' SELECT * FROM t WHERE a != 1; +id estRows task access object operator info +TableReader 6.00 root partition:all data:Selection +└─Selection 6.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, 1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a != 1; +a b +0 0 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1; +id estRows task access object operator info +TableReader 5.00 root partition:all data:Selection +└─Selection 5.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, -1), ne(executor__partition__partition_boundaries.t.a, 0), ne(executor__partition__partition_boundaries.t.a, 1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1); +id estRows task access object operator info +TableReader 5.00 root partition:all data:Selection +└─Selection 5.00 cop[tikv] not(in(executor__partition__partition_boundaries.t.a, -2, -1, 0, 1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1); +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1; +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] or(or(0, eq(executor__partition__partition_boundaries.t.a, -1)), or(eq(executor__partition__partition_boundaries.t.a, 0), eq(executor__partition__partition_boundaries.t.a, 1))) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1; +a b +0 0 Filler... +1 1 Filler... +explain format='brief' SELECT * FROM t WHERE a != 2; +id estRows task access object operator info +TableReader 6.00 root partition:all data:Selection +└─Selection 6.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, 2) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a != 2; +a b +0 0 Filler... +1 1 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2; +id estRows task access object operator info +TableReader 4.00 root partition:all data:Selection +└─Selection 4.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, -1), ne(executor__partition__partition_boundaries.t.a, 0), ne(executor__partition__partition_boundaries.t.a, 1), ne(executor__partition__partition_boundaries.t.a, 2) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2; +a b +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2); +id estRows task access object operator info +TableReader 4.00 root partition:all data:Selection +└─Selection 4.00 cop[tikv] not(in(executor__partition__partition_boundaries.t.a, -2, -1, 0, 1, 2)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2); +a b +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2; +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1,p2 data:Selection +└─Selection 3.00 cop[tikv] or(or(0, eq(executor__partition__partition_boundaries.t.a, -1)), or(eq(executor__partition__partition_boundaries.t.a, 0), or(eq(executor__partition__partition_boundaries.t.a, 1), eq(executor__partition__partition_boundaries.t.a, 2)))) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +explain format='brief' SELECT * FROM t WHERE a != 3; +id estRows task access object operator info +TableReader 6.00 root partition:all data:Selection +└─Selection 6.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, 3) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a != 3; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3; +id estRows task access object operator info +TableReader 3.00 root partition:all data:Selection +└─Selection 3.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, -1), ne(executor__partition__partition_boundaries.t.a, 0), ne(executor__partition__partition_boundaries.t.a, 1), ne(executor__partition__partition_boundaries.t.a, 2), ne(executor__partition__partition_boundaries.t.a, 3) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3; +a b +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3); +id estRows task access object operator info +TableReader 3.00 root partition:all data:Selection +└─Selection 3.00 cop[tikv] not(in(executor__partition__partition_boundaries.t.a, -2, -1, 0, 1, 2, 3)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3); +a b +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3; +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1,p2,p3 data:Selection +└─Selection 4.00 cop[tikv] or(or(0, or(eq(executor__partition__partition_boundaries.t.a, -1), eq(executor__partition__partition_boundaries.t.a, 0))), or(eq(executor__partition__partition_boundaries.t.a, 1), or(eq(executor__partition__partition_boundaries.t.a, 2), eq(executor__partition__partition_boundaries.t.a, 3)))) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +explain format='brief' SELECT * FROM t WHERE a != 4; +id estRows task access object operator info +TableReader 6.00 root partition:all data:Selection +└─Selection 6.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a != 4; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4; +id estRows task access object operator info +TableReader 2.00 root partition:all data:Selection +└─Selection 2.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, -1), ne(executor__partition__partition_boundaries.t.a, 0), ne(executor__partition__partition_boundaries.t.a, 1), ne(executor__partition__partition_boundaries.t.a, 2), ne(executor__partition__partition_boundaries.t.a, 3), ne(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4; +a b +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4); +id estRows task access object operator info +TableReader 2.00 root partition:all data:Selection +└─Selection 2.00 cop[tikv] not(in(executor__partition__partition_boundaries.t.a, -2, -1, 0, 1, 2, 3, 4)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4); +a b +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1,p2,p3,p4 data:Selection +└─Selection 5.00 cop[tikv] or(or(0, or(eq(executor__partition__partition_boundaries.t.a, -1), eq(executor__partition__partition_boundaries.t.a, 0))), or(or(eq(executor__partition__partition_boundaries.t.a, 1), eq(executor__partition__partition_boundaries.t.a, 2)), or(eq(executor__partition__partition_boundaries.t.a, 3), eq(executor__partition__partition_boundaries.t.a, 4)))) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE a != 5; +id estRows task access object operator info +TableReader 6.00 root partition:all data:Selection +└─Selection 6.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, 5) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a != 5; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4 AND a != 5; +id estRows task access object operator info +TableReader 1.00 root partition:all data:Selection +└─Selection 1.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, -1), ne(executor__partition__partition_boundaries.t.a, 0), ne(executor__partition__partition_boundaries.t.a, 1), ne(executor__partition__partition_boundaries.t.a, 2), ne(executor__partition__partition_boundaries.t.a, 3), ne(executor__partition__partition_boundaries.t.a, 4), ne(executor__partition__partition_boundaries.t.a, 5) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4 AND a != 5; +a b +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4, 5); +id estRows task access object operator info +TableReader 1.00 root partition:all data:Selection +└─Selection 1.00 cop[tikv] not(in(executor__partition__partition_boundaries.t.a, -2, -1, 0, 1, 2, 3, 4, 5)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4, 5); +a b +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4 OR a = 5; +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1,p2,p3,p4,p5 data:Selection +└─Selection 6.00 cop[tikv] or(or(or(0, eq(executor__partition__partition_boundaries.t.a, -1)), or(eq(executor__partition__partition_boundaries.t.a, 0), eq(executor__partition__partition_boundaries.t.a, 1))), or(or(eq(executor__partition__partition_boundaries.t.a, 2), eq(executor__partition__partition_boundaries.t.a, 3)), or(eq(executor__partition__partition_boundaries.t.a, 4), eq(executor__partition__partition_boundaries.t.a, 5)))) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4 OR a = 5; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +explain format='brief' SELECT * FROM t WHERE a != 6; +id estRows task access object operator info +TableReader 6.00 root partition:all data:Selection +└─Selection 6.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, 6) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a != 6; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4 AND a != 5 AND a != 6; +id estRows task access object operator info +TableReader 1.00 root partition:all data:Selection +└─Selection 1.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, -1), ne(executor__partition__partition_boundaries.t.a, 0), ne(executor__partition__partition_boundaries.t.a, 1), ne(executor__partition__partition_boundaries.t.a, 2), ne(executor__partition__partition_boundaries.t.a, 3), ne(executor__partition__partition_boundaries.t.a, 4), ne(executor__partition__partition_boundaries.t.a, 5), ne(executor__partition__partition_boundaries.t.a, 6) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4 AND a != 5 AND a != 6; +a b +explain format='brief' SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4, 5, 6); +id estRows task access object operator info +TableReader 1.00 root partition:all data:Selection +└─Selection 1.00 cop[tikv] not(in(executor__partition__partition_boundaries.t.a, -2, -1, 0, 1, 2, 3, 4, 5, 6)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4, 5, 6); +a b +explain format='brief' SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4 OR a = 5 OR a = 6; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(or(or(0, eq(executor__partition__partition_boundaries.t.a, -1)), or(eq(executor__partition__partition_boundaries.t.a, 0), eq(executor__partition__partition_boundaries.t.a, 1))), or(or(eq(executor__partition__partition_boundaries.t.a, 2), eq(executor__partition__partition_boundaries.t.a, 3)), or(eq(executor__partition__partition_boundaries.t.a, 4), or(eq(executor__partition__partition_boundaries.t.a, 5), eq(executor__partition__partition_boundaries.t.a, 6))))) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4 OR a = 5 OR a = 6; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a != 7; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, 7) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a != 7; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4 AND a != 5 AND a != 6 AND a != 7; +id estRows task access object operator info +TableReader 1.00 root partition:all data:Selection +└─Selection 1.00 cop[tikv] ne(executor__partition__partition_boundaries.t.a, -1), ne(executor__partition__partition_boundaries.t.a, 0), ne(executor__partition__partition_boundaries.t.a, 1), ne(executor__partition__partition_boundaries.t.a, 2), ne(executor__partition__partition_boundaries.t.a, 3), ne(executor__partition__partition_boundaries.t.a, 4), ne(executor__partition__partition_boundaries.t.a, 5), ne(executor__partition__partition_boundaries.t.a, 6), ne(executor__partition__partition_boundaries.t.a, 7) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4 AND a != 5 AND a != 6 AND a != 7; +a b +explain format='brief' SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4, 5, 6, 7); +id estRows task access object operator info +TableReader 1.00 root partition:all data:Selection +└─Selection 1.00 cop[tikv] not(in(executor__partition__partition_boundaries.t.a, -2, -1, 0, 1, 2, 3, 4, 5, 6, 7)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4, 5, 6, 7); +a b +explain format='brief' SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4 OR a = 5 OR a = 6 OR a = 7; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(or(or(0, eq(executor__partition__partition_boundaries.t.a, -1)), or(eq(executor__partition__partition_boundaries.t.a, 0), or(eq(executor__partition__partition_boundaries.t.a, 1), eq(executor__partition__partition_boundaries.t.a, 2)))), or(or(eq(executor__partition__partition_boundaries.t.a, 3), eq(executor__partition__partition_boundaries.t.a, 4)), or(eq(executor__partition__partition_boundaries.t.a, 5), or(eq(executor__partition__partition_boundaries.t.a, 6), eq(executor__partition__partition_boundaries.t.a, 7))))) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4 OR a = 5 OR a = 6 OR a = 7; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +SET @@tidb_partition_prune_mode = default; +DROP TABLE IF EXISTS t; +CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( +PARTITION p0 VALUES LESS THAN (1000000), +PARTITION p1 VALUES LESS THAN (2000000), +PARTITION p2 VALUES LESS THAN (3000000)); +INSERT INTO t VALUES (999998, '999998 Filler ...'), (999999, '999999 Filler ...'), (1000000, '1000000 Filler ...'), (1000001, '1000001 Filler ...'), (1000002, '1000002 Filler ...'); +INSERT INTO t VALUES (1999998, '1999998 Filler ...'), (1999999, '1999999 Filler ...'), (2000000, '2000000 Filler ...'), (2000001, '2000001 Filler ...'), (2000002, '2000002 Filler ...'); +INSERT INTO t VALUES (2999998, '2999998 Filler ...'), (2999999, '2999999 Filler ...'); +INSERT INTO t VALUES (-2147483648, 'MIN_INT filler...'), (0, '0 Filler...'); +ANALYZE TABLE t all columns; +explain format='brief' SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483649; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, -2147483648), le(executor__partition__partition_boundaries.t.a, -2147483649) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483649; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483648; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, -2147483648), le(executor__partition__partition_boundaries.t.a, -2147483648) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483648; +a b +-2147483648 MIN_INT filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483647; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, -2147483648), le(executor__partition__partition_boundaries.t.a, -2147483647) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483647; +a b +-2147483648 MIN_INT filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483646; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, -2147483648), le(executor__partition__partition_boundaries.t.a, -2147483646) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483646; +a b +-2147483648 MIN_INT filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483638; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, -2147483648), le(executor__partition__partition_boundaries.t.a, -2147483638) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483638; +a b +-2147483648 MIN_INT filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483650; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, -2147483648), le(executor__partition__partition_boundaries.t.a, -2146483650) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483650; +a b +-2147483648 MIN_INT filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483649; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, -2147483648), le(executor__partition__partition_boundaries.t.a, -2146483649) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483649; +a b +-2147483648 MIN_INT filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483648; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, -2147483648), le(executor__partition__partition_boundaries.t.a, -2146483648) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483648; +a b +-2147483648 MIN_INT filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483647; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, -2147483648), le(executor__partition__partition_boundaries.t.a, -2146483647) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483647; +a b +-2147483648 MIN_INT filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483646; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, -2147483648), le(executor__partition__partition_boundaries.t.a, -2146483646) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483646; +a b +-2147483648 MIN_INT filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 0 AND -1; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 0), le(executor__partition__partition_boundaries.t.a, -1) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 0 AND -1; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 0 AND 0; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 0), le(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 0 AND 0; +a b +0 0 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 0 AND 1; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 0), le(executor__partition__partition_boundaries.t.a, 1) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 0 AND 1; +a b +0 0 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 0 AND 2; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 0), le(executor__partition__partition_boundaries.t.a, 2) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 0 AND 2; +a b +0 0 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 0 AND 10; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 0), le(executor__partition__partition_boundaries.t.a, 10) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 0 AND 10; +a b +0 0 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 0 AND 999998; +id estRows task access object operator info +TableReader 2.00 root partition:p0 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 0), le(executor__partition__partition_boundaries.t.a, 999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 0 AND 999998; +a b +0 0 Filler... +999998 999998 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 0 AND 999999; +id estRows task access object operator info +TableReader 3.00 root partition:p0 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 0), le(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 0 AND 999999; +a b +0 0 Filler... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 0 AND 1000000; +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1 data:Selection +└─Selection 4.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 0), le(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 0 AND 1000000; +a b +0 0 Filler... +1000000 1000000 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 0 AND 1000001; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 0), le(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 0 AND 1000001; +a b +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 0 AND 1000002; +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1 data:Selection +└─Selection 6.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 0), le(executor__partition__partition_boundaries.t.a, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 0 AND 1000002; +a b +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999998 AND 999997; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999998), le(executor__partition__partition_boundaries.t.a, 999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999998 AND 999997; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999998 AND 999998; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999998), le(executor__partition__partition_boundaries.t.a, 999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999998 AND 999998; +a b +999998 999998 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999998 AND 999999; +id estRows task access object operator info +TableReader 2.00 root partition:p0 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999998), le(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999998 AND 999999; +a b +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999998 AND 1000000; +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999998), le(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999998 AND 1000000; +a b +1000000 1000000 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999998 AND 1000008; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999998), le(executor__partition__partition_boundaries.t.a, 1000008) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999998 AND 1000008; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999998 AND 1999996; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999998), le(executor__partition__partition_boundaries.t.a, 1999996) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999998 AND 1999996; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999998 AND 1999997; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999998), le(executor__partition__partition_boundaries.t.a, 1999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999998 AND 1999997; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999998 AND 1999998; +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1 data:Selection +└─Selection 6.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999998), le(executor__partition__partition_boundaries.t.a, 1999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999998 AND 1999998; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999998 AND 1999999; +id estRows task access object operator info +TableReader 7.00 root partition:p0,p1 data:Selection +└─Selection 7.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999998), le(executor__partition__partition_boundaries.t.a, 1999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999998 AND 1999999; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999998 AND 2000000; +id estRows task access object operator info +TableReader 8.00 root partition:all data:Selection +└─Selection 8.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999998), le(executor__partition__partition_boundaries.t.a, 2000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999998 AND 2000000; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999999 AND 999998; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999999), le(executor__partition__partition_boundaries.t.a, 999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999999 AND 999998; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999999 AND 999999; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999999), le(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999999 AND 999999; +a b +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999999 AND 1000000; +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999999), le(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999999 AND 1000000; +a b +1000000 1000000 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999999 AND 1000001; +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999999), le(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999999 AND 1000001; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999999 AND 1000009; +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1 data:Selection +└─Selection 4.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999999), le(executor__partition__partition_boundaries.t.a, 1000009) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999999 AND 1000009; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999999 AND 1999997; +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1 data:Selection +└─Selection 4.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999999), le(executor__partition__partition_boundaries.t.a, 1999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999999 AND 1999997; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999999 AND 1999998; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999999), le(executor__partition__partition_boundaries.t.a, 1999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999999 AND 1999998; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999999 AND 1999999; +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1 data:Selection +└─Selection 6.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999999), le(executor__partition__partition_boundaries.t.a, 1999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999999 AND 1999999; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999999 AND 2000000; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999999), le(executor__partition__partition_boundaries.t.a, 2000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999999 AND 2000000; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 999999 AND 2000001; +id estRows task access object operator info +TableReader 8.00 root partition:all data:Selection +└─Selection 8.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999999), le(executor__partition__partition_boundaries.t.a, 2000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 999999 AND 2000001; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000000 AND 999999; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000000), le(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000000 AND 999999; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000000; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000000), le(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000000; +a b +1000000 1000000 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000001; +id estRows task access object operator info +TableReader 2.00 root partition:p1 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000000), le(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000001; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000002; +id estRows task access object operator info +TableReader 3.00 root partition:p1 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000000), le(executor__partition__partition_boundaries.t.a, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000002; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000010; +id estRows task access object operator info +TableReader 3.00 root partition:p1 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000000), le(executor__partition__partition_boundaries.t.a, 1000010) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000010; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000000 AND 1999998; +id estRows task access object operator info +TableReader 4.00 root partition:p1 data:Selection +└─Selection 4.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000000), le(executor__partition__partition_boundaries.t.a, 1999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000000 AND 1999998; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000000 AND 1999999; +id estRows task access object operator info +TableReader 5.00 root partition:p1 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000000), le(executor__partition__partition_boundaries.t.a, 1999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000000 AND 1999999; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000000 AND 2000000; +id estRows task access object operator info +TableReader 6.00 root partition:p1,p2 data:Selection +└─Selection 6.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000000), le(executor__partition__partition_boundaries.t.a, 2000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000000 AND 2000000; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000000 AND 2000001; +id estRows task access object operator info +TableReader 7.00 root partition:p1,p2 data:Selection +└─Selection 7.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000000), le(executor__partition__partition_boundaries.t.a, 2000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000000 AND 2000001; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000000 AND 2000002; +id estRows task access object operator info +TableReader 8.00 root partition:p1,p2 data:Selection +└─Selection 8.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000000), le(executor__partition__partition_boundaries.t.a, 2000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000000 AND 2000002; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000000; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000001), le(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000000; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000001; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000001), le(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000001; +a b +1000001 1000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000002; +id estRows task access object operator info +TableReader 2.00 root partition:p1 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000001), le(executor__partition__partition_boundaries.t.a, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000002; +a b +1000001 1000001 Filler ... +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000003; +id estRows task access object operator info +TableReader 2.00 root partition:p1 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000001), le(executor__partition__partition_boundaries.t.a, 1000003) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000003; +a b +1000001 1000001 Filler ... +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000011; +id estRows task access object operator info +TableReader 2.00 root partition:p1 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000001), le(executor__partition__partition_boundaries.t.a, 1000011) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000011; +a b +1000001 1000001 Filler ... +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000001 AND 1999999; +id estRows task access object operator info +TableReader 4.00 root partition:p1 data:Selection +└─Selection 4.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000001), le(executor__partition__partition_boundaries.t.a, 1999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000001 AND 1999999; +a b +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000000; +id estRows task access object operator info +TableReader 5.00 root partition:p1,p2 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000001), le(executor__partition__partition_boundaries.t.a, 2000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000000; +a b +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000001; +id estRows task access object operator info +TableReader 6.00 root partition:p1,p2 data:Selection +└─Selection 6.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000001), le(executor__partition__partition_boundaries.t.a, 2000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000001; +a b +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000002; +id estRows task access object operator info +TableReader 7.00 root partition:p1,p2 data:Selection +└─Selection 7.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000001), le(executor__partition__partition_boundaries.t.a, 2000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000002; +a b +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000003; +id estRows task access object operator info +TableReader 7.00 root partition:p1,p2 data:Selection +└─Selection 7.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000001), le(executor__partition__partition_boundaries.t.a, 2000003) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000003; +a b +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000001; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000002), le(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000001; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000002; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000002), le(executor__partition__partition_boundaries.t.a, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000002; +a b +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000003; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000002), le(executor__partition__partition_boundaries.t.a, 1000003) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000003; +a b +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000004; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000002), le(executor__partition__partition_boundaries.t.a, 1000004) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000004; +a b +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000012; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000002), le(executor__partition__partition_boundaries.t.a, 1000012) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000012; +a b +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000000; +id estRows task access object operator info +TableReader 4.00 root partition:p1,p2 data:Selection +└─Selection 4.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000002), le(executor__partition__partition_boundaries.t.a, 2000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000000; +a b +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000001; +id estRows task access object operator info +TableReader 5.00 root partition:p1,p2 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000002), le(executor__partition__partition_boundaries.t.a, 2000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000001; +a b +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000002; +id estRows task access object operator info +TableReader 6.00 root partition:p1,p2 data:Selection +└─Selection 6.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000002), le(executor__partition__partition_boundaries.t.a, 2000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000002; +a b +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000003; +id estRows task access object operator info +TableReader 6.00 root partition:p1,p2 data:Selection +└─Selection 6.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000002), le(executor__partition__partition_boundaries.t.a, 2000003) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000003; +a b +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000004; +id estRows task access object operator info +TableReader 6.00 root partition:p1,p2 data:Selection +└─Selection 6.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000002), le(executor__partition__partition_boundaries.t.a, 2000004) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000004; +a b +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000000 AND 2999999; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000000), le(executor__partition__partition_boundaries.t.a, 2999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000000 AND 2999999; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000000; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000000), le(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000000; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000001; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000000), le(executor__partition__partition_boundaries.t.a, 3000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000001; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000002; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000000), le(executor__partition__partition_boundaries.t.a, 3000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000002; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000010; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000000), le(executor__partition__partition_boundaries.t.a, 3000010) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000010; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000000 AND 3999998; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000000), le(executor__partition__partition_boundaries.t.a, 3999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000000 AND 3999998; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000000 AND 3999999; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000000), le(executor__partition__partition_boundaries.t.a, 3999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000000 AND 3999999; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000000 AND 4000000; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000000), le(executor__partition__partition_boundaries.t.a, 4000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000000 AND 4000000; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000000 AND 4000001; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000000), le(executor__partition__partition_boundaries.t.a, 4000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000000 AND 4000001; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000000 AND 4000002; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000000), le(executor__partition__partition_boundaries.t.a, 4000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000000 AND 4000002; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000000; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000001), le(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000000; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000001; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000001), le(executor__partition__partition_boundaries.t.a, 3000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000001; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000002; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000001), le(executor__partition__partition_boundaries.t.a, 3000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000002; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000003; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000001), le(executor__partition__partition_boundaries.t.a, 3000003) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000003; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000011; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000001), le(executor__partition__partition_boundaries.t.a, 3000011) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000011; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000001 AND 3999999; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000001), le(executor__partition__partition_boundaries.t.a, 3999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000001 AND 3999999; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000000; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000001), le(executor__partition__partition_boundaries.t.a, 4000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000000; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000001; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000001), le(executor__partition__partition_boundaries.t.a, 4000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000001; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000002; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000001), le(executor__partition__partition_boundaries.t.a, 4000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000002; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000003; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000001), le(executor__partition__partition_boundaries.t.a, 4000003) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000003; +a b +DROP TABLE IF EXISTS t; +CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( +PARTITION p0 VALUES LESS THAN (1), +PARTITION p1 VALUES LESS THAN (2), +PARTITION p2 VALUES LESS THAN (3), +PARTITION p3 VALUES LESS THAN (4), +PARTITION p4 VALUES LESS THAN (5), +PARTITION p5 VALUES LESS THAN (6), +PARTITION p6 VALUES LESS THAN (7)); +INSERT INTO t VALUES (0, '0 Filler...'); +INSERT INTO t VALUES (1, '1 Filler...'); +INSERT INTO t VALUES (2, '2 Filler...'); +INSERT INTO t VALUES (3, '3 Filler...'); +INSERT INTO t VALUES (4, '4 Filler...'); +INSERT INTO t VALUES (5, '5 Filler...'); +INSERT INTO t VALUES (6, '6 Filler...'); +ANALYZE TABLE t all columns; +explain format='brief' SELECT * FROM t WHERE a BETWEEN 2 AND -1; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, -1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 2 AND -1; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN -1 AND 4; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1,p2,p3,p4 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, -1), le(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN -1 AND 4; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 2 AND 0; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 2 AND 0; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 0 AND 4; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1,p2,p3,p4 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 0), le(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 0 AND 4; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 2 AND 1; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 2 AND 1; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 1 AND 4; +id estRows task access object operator info +TableReader 4.00 root partition:p1,p2,p3,p4 data:Selection +└─Selection 4.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1), le(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 1 AND 4; +a b +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 2 AND 2; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 2) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 2 AND 2; +a b +2 2 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 2 AND 4; +id estRows task access object operator info +TableReader 3.00 root partition:p2,p3,p4 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 2 AND 4; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 2 AND 3; +id estRows task access object operator info +TableReader 2.00 root partition:p2,p3 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 3) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 2 AND 3; +a b +2 2 Filler... +3 3 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 3 AND 4; +id estRows task access object operator info +TableReader 2.00 root partition:p3,p4 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3), le(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 3 AND 4; +a b +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 2 AND 4; +id estRows task access object operator info +TableReader 3.00 root partition:p2,p3,p4 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 2 AND 4; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 4 AND 4; +id estRows task access object operator info +TableReader 1.00 root partition:p4 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 4), le(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 4 AND 4; +a b +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 2 AND 5; +id estRows task access object operator info +TableReader 4.00 root partition:p2,p3,p4,p5 data:Selection +└─Selection 4.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 5) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 2 AND 5; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 5 AND 4; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 5), le(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 5 AND 4; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 2 AND 6; +id estRows task access object operator info +TableReader 5.00 root partition:p2,p3,p4,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 6) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 2 AND 6; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 6 AND 4; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 6), le(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 6 AND 4; +a b +explain format='brief' SELECT * FROM t WHERE a BETWEEN 2 AND 7; +id estRows task access object operator info +TableReader 5.00 root partition:p2,p3,p4,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 7) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 2 AND 7; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a BETWEEN 7 AND 4; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 7), le(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a BETWEEN 7 AND 4; +a b +set @@tidb_partition_prune_mode = 'dynamic'; +drop table if exists t; +CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( +PARTITION p0 VALUES LESS THAN (1000000), +PARTITION p1 VALUES LESS THAN (2000000), +PARTITION p2 VALUES LESS THAN (3000000)); +INSERT INTO t VALUES (999998, '999998 Filler ...'), (999999, '999999 Filler ...'), (1000000, '1000000 Filler ...'), (1000001, '1000001 Filler ...'), (1000002, '1000002 Filler ...'); +INSERT INTO t VALUES (1999998, '1999998 Filler ...'), (1999999, '1999999 Filler ...'), (2000000, '2000000 Filler ...'), (2000001, '2000001 Filler ...'), (2000002, '2000002 Filler ...'); +INSERT INTO t VALUES (2999998, '2999998 Filler ...'), (2999999, '2999999 Filler ...'); +INSERT INTO t VALUES (-2147483648, 'MIN_INT filler...'), (0, '0 Filler...'); +ANALYZE TABLE t all columns; +explain format='brief' SELECT * FROM t WHERE a < -2147483648; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, -2147483648) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < -2147483648; +a b +explain format='brief' SELECT * FROM t WHERE a > -2147483648; +id estRows task access object operator info +TableReader 13.00 root partition:all data:Selection +└─Selection 13.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, -2147483648) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > -2147483648; +a b +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= -2147483648; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, -2147483648) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= -2147483648; +a b +-2147483648 MIN_INT filler... +explain format='brief' SELECT * FROM t WHERE a >= -2147483648; +id estRows task access object operator info +TableReader 14.00 root partition:all data:Selection +└─Selection 14.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, -2147483648) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= -2147483648; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 0; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 0; +a b +-2147483648 MIN_INT filler... +explain format='brief' SELECT * FROM t WHERE a > 0; +id estRows task access object operator info +TableReader 12.00 root partition:all data:Selection +└─Selection 12.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 0; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 0; +id estRows task access object operator info +TableReader 2.00 root partition:p0 data:Selection +└─Selection 2.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 0; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 0; +id estRows task access object operator info +TableReader 13.00 root partition:all data:Selection +└─Selection 13.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 0; +a b +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 999998; +id estRows task access object operator info +TableReader 2.00 root partition:p0 data:Selection +└─Selection 2.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 999998; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +explain format='brief' SELECT * FROM t WHERE a > 999998; +id estRows task access object operator info +TableReader 11.00 root partition:all data:Selection +└─Selection 11.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 999998; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 999998; +id estRows task access object operator info +TableReader 3.00 root partition:p0 data:Selection +└─Selection 3.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 999998; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +999998 999998 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 999998; +id estRows task access object operator info +TableReader 12.00 root partition:all data:Selection +└─Selection 12.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 999998; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 999999; +id estRows task access object operator info +TableReader 3.00 root partition:p0 data:Selection +└─Selection 3.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 999999; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +999998 999998 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 999999; +id estRows task access object operator info +TableReader 10.00 root partition:p1,p2 data:Selection +└─Selection 10.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 999999; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 999999; +id estRows task access object operator info +TableReader 4.00 root partition:p0 data:Selection +└─Selection 4.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 999999; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 999999; +id estRows task access object operator info +TableReader 11.00 root partition:all data:Selection +└─Selection 11.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 999999; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 1000000; +id estRows task access object operator info +TableReader 4.00 root partition:p0 data:Selection +└─Selection 4.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 1000000; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1000000; +id estRows task access object operator info +TableReader 9.00 root partition:p1,p2 data:Selection +└─Selection 9.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1000000; +a b +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 1000000; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1 data:Selection +└─Selection 5.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 1000000; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 1000000; +id estRows task access object operator info +TableReader 10.00 root partition:p1,p2 data:Selection +└─Selection 10.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 1000000; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 1000001; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1 data:Selection +└─Selection 5.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 1000001; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1000001; +id estRows task access object operator info +TableReader 8.00 root partition:p1,p2 data:Selection +└─Selection 8.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1000001; +a b +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 1000001; +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1 data:Selection +└─Selection 6.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 1000001; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 1000001; +id estRows task access object operator info +TableReader 9.00 root partition:p1,p2 data:Selection +└─Selection 9.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 1000001; +a b +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 1000002; +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1 data:Selection +└─Selection 6.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 1000002; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1000002; +id estRows task access object operator info +TableReader 7.00 root partition:p1,p2 data:Selection +└─Selection 7.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1000002; +a b +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 1000002; +id estRows task access object operator info +TableReader 7.00 root partition:p0,p1 data:Selection +└─Selection 7.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 1000002; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 1000002; +id estRows task access object operator info +TableReader 8.00 root partition:p1,p2 data:Selection +└─Selection 8.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 1000002; +a b +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 3000000; +id estRows task access object operator info +TableReader 14.00 root partition:all data:Selection +└─Selection 14.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 3000000; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 3000000; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 3000000; +a b +explain format='brief' SELECT * FROM t WHERE a <= 3000000; +id estRows task access object operator info +TableReader 14.00 root partition:all data:Selection +└─Selection 14.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 3000000; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 3000000; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 3000000; +a b +explain format='brief' SELECT * FROM t WHERE a < 3000001; +id estRows task access object operator info +TableReader 14.00 root partition:all data:Selection +└─Selection 14.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 3000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 3000001; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 3000001; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 3000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 3000001; +a b +explain format='brief' SELECT * FROM t WHERE a <= 3000001; +id estRows task access object operator info +TableReader 14.00 root partition:all data:Selection +└─Selection 14.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 3000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 3000001; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 3000001; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 3000001; +a b +explain format='brief' SELECT * FROM t WHERE a < 999997; +id estRows task access object operator info +TableReader 2.00 root partition:p0 data:Selection +└─Selection 2.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 999997; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +explain format='brief' SELECT * FROM t WHERE a > 999997; +id estRows task access object operator info +TableReader 12.00 root partition:all data:Selection +└─Selection 12.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 999997; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 999997; +id estRows task access object operator info +TableReader 2.00 root partition:p0 data:Selection +└─Selection 2.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 999997; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 999997; +id estRows task access object operator info +TableReader 12.00 root partition:all data:Selection +└─Selection 12.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 999997; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 999997 AND a <= 999999; +id estRows task access object operator info +TableReader 2.00 root partition:p0 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999997), le(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 999997 AND a <= 999999; +a b +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 999997 AND a <= 999999; +id estRows task access object operator info +TableReader 2.00 root partition:p0 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 999997), le(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 999997 AND a <= 999999; +a b +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 999997 AND a < 999999; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 999997), lt(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 999997 AND a < 999999; +a b +999998 999998 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 999997 AND a <= 999999; +id estRows task access object operator info +TableReader 2.00 root partition:p0 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 999997), le(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 999997 AND a <= 999999; +a b +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 999998; +id estRows task access object operator info +TableReader 2.00 root partition:p0 data:Selection +└─Selection 2.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 999998; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +explain format='brief' SELECT * FROM t WHERE a > 999998; +id estRows task access object operator info +TableReader 11.00 root partition:all data:Selection +└─Selection 11.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 999998; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 999998; +id estRows task access object operator info +TableReader 3.00 root partition:p0 data:Selection +└─Selection 3.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 999998; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +999998 999998 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 999998; +id estRows task access object operator info +TableReader 12.00 root partition:all data:Selection +└─Selection 12.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 999998; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 999998 AND a <= 1000000; +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999998), le(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 999998 AND a <= 1000000; +a b +1000000 1000000 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 999998 AND a <= 1000000; +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 999998), le(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 999998 AND a <= 1000000; +a b +1000000 1000000 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 999998 AND a < 1000000; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 999998), lt(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 999998 AND a < 1000000; +a b +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 999998 AND a <= 1000000; +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 999998), le(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 999998 AND a <= 1000000; +a b +1000000 1000000 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 999999; +id estRows task access object operator info +TableReader 3.00 root partition:p0 data:Selection +└─Selection 3.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 999999; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +999998 999998 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 999999; +id estRows task access object operator info +TableReader 10.00 root partition:p1,p2 data:Selection +└─Selection 10.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 999999; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 999999; +id estRows task access object operator info +TableReader 4.00 root partition:p0 data:Selection +└─Selection 4.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 999999; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 999999; +id estRows task access object operator info +TableReader 11.00 root partition:all data:Selection +└─Selection 11.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 999999; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 999999 AND a <= 1000001; +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 999999), le(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 999999 AND a <= 1000001; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 999999 AND a <= 1000001; +id estRows task access object operator info +TableReader 2.00 root partition:p1 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 999999), le(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 999999 AND a <= 1000001; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 999999 AND a < 1000001; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 999999), lt(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 999999 AND a < 1000001; +a b +1000000 1000000 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 999999 AND a <= 1000001; +id estRows task access object operator info +TableReader 2.00 root partition:p1 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 999999), le(executor__partition__partition_boundaries.t.a, 1000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 999999 AND a <= 1000001; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 1000000; +id estRows task access object operator info +TableReader 4.00 root partition:p0 data:Selection +└─Selection 4.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 1000000; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1000000; +id estRows task access object operator info +TableReader 9.00 root partition:p1,p2 data:Selection +└─Selection 9.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1000000; +a b +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 1000000; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1 data:Selection +└─Selection 5.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 1000000; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 1000000; +id estRows task access object operator info +TableReader 10.00 root partition:p1,p2 data:Selection +└─Selection 10.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 1000000; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 1000000 AND a <= 1000002; +id estRows task access object operator info +TableReader 3.00 root partition:p1 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1000000), le(executor__partition__partition_boundaries.t.a, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 1000000 AND a <= 1000002; +a b +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1000000 AND a <= 1000002; +id estRows task access object operator info +TableReader 2.00 root partition:p1 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1000000), le(executor__partition__partition_boundaries.t.a, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1000000 AND a <= 1000002; +a b +1000001 1000001 Filler ... +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1000000 AND a < 1000002; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1000000), lt(executor__partition__partition_boundaries.t.a, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1000000 AND a < 1000002; +a b +1000001 1000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1000000 AND a <= 1000002; +id estRows task access object operator info +TableReader 2.00 root partition:p1 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1000000), le(executor__partition__partition_boundaries.t.a, 1000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1000000 AND a <= 1000002; +a b +1000001 1000001 Filler ... +1000002 1000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 1999997; +id estRows task access object operator info +TableReader 7.00 root partition:p0,p1 data:Selection +└─Selection 7.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 1999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 1999997; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1999997; +id estRows task access object operator info +TableReader 7.00 root partition:p1,p2 data:Selection +└─Selection 7.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1999997; +a b +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 1999997; +id estRows task access object operator info +TableReader 7.00 root partition:p0,p1 data:Selection +└─Selection 7.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 1999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 1999997; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 1999997; +id estRows task access object operator info +TableReader 7.00 root partition:p1,p2 data:Selection +└─Selection 7.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 1999997; +a b +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 1999997 AND a <= 1999999; +id estRows task access object operator info +TableReader 2.00 root partition:p1 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1999997), le(executor__partition__partition_boundaries.t.a, 1999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 1999997 AND a <= 1999999; +a b +1999998 1999998 Filler ... +1999999 1999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1999997 AND a <= 1999999; +id estRows task access object operator info +TableReader 2.00 root partition:p1 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1999997), le(executor__partition__partition_boundaries.t.a, 1999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1999997 AND a <= 1999999; +a b +1999998 1999998 Filler ... +1999999 1999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1999997 AND a < 1999999; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1999997), lt(executor__partition__partition_boundaries.t.a, 1999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1999997 AND a < 1999999; +a b +1999998 1999998 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1999997 AND a <= 1999999; +id estRows task access object operator info +TableReader 2.00 root partition:p1 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1999997), le(executor__partition__partition_boundaries.t.a, 1999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1999997 AND a <= 1999999; +a b +1999998 1999998 Filler ... +1999999 1999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 1999998; +id estRows task access object operator info +TableReader 7.00 root partition:p0,p1 data:Selection +└─Selection 7.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 1999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 1999998; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1999998; +id estRows task access object operator info +TableReader 6.00 root partition:p1,p2 data:Selection +└─Selection 6.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1999998; +a b +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 1999998; +id estRows task access object operator info +TableReader 8.00 root partition:p0,p1 data:Selection +└─Selection 8.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 1999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 1999998; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 1999998; +id estRows task access object operator info +TableReader 7.00 root partition:p1,p2 data:Selection +└─Selection 7.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 1999998; +a b +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 1999998 AND a <= 2000000; +id estRows task access object operator info +TableReader 3.00 root partition:p1,p2 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1999998), le(executor__partition__partition_boundaries.t.a, 2000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 1999998 AND a <= 2000000; +a b +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1999998 AND a <= 2000000; +id estRows task access object operator info +TableReader 2.00 root partition:p1,p2 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1999998), le(executor__partition__partition_boundaries.t.a, 2000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1999998 AND a <= 2000000; +a b +1999999 1999999 Filler ... +2000000 2000000 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1999998 AND a < 2000000; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1999998), lt(executor__partition__partition_boundaries.t.a, 2000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1999998 AND a < 2000000; +a b +1999999 1999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1999998 AND a <= 2000000; +id estRows task access object operator info +TableReader 2.00 root partition:p1,p2 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1999998), le(executor__partition__partition_boundaries.t.a, 2000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1999998 AND a <= 2000000; +a b +1999999 1999999 Filler ... +2000000 2000000 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 1999999; +id estRows task access object operator info +TableReader 8.00 root partition:p0,p1 data:Selection +└─Selection 8.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 1999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 1999999; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1999999; +id estRows task access object operator info +TableReader 5.00 root partition:p2 data:Selection +└─Selection 5.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1999999; +a b +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 1999999; +id estRows task access object operator info +TableReader 9.00 root partition:p0,p1 data:Selection +└─Selection 9.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 1999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 1999999; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 1999999; +id estRows task access object operator info +TableReader 6.00 root partition:p1,p2 data:Selection +└─Selection 6.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 1999999; +a b +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 1999999 AND a <= 2000001; +id estRows task access object operator info +TableReader 3.00 root partition:p1,p2 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1999999), le(executor__partition__partition_boundaries.t.a, 2000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 1999999 AND a <= 2000001; +a b +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1999999 AND a <= 2000001; +id estRows task access object operator info +TableReader 2.00 root partition:p2 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1999999), le(executor__partition__partition_boundaries.t.a, 2000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1999999 AND a <= 2000001; +a b +2000000 2000000 Filler ... +2000001 2000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1999999 AND a < 2000001; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1999999), lt(executor__partition__partition_boundaries.t.a, 2000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1999999 AND a < 2000001; +a b +2000000 2000000 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 1999999 AND a <= 2000001; +id estRows task access object operator info +TableReader 2.00 root partition:p2 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1999999), le(executor__partition__partition_boundaries.t.a, 2000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1999999 AND a <= 2000001; +a b +2000000 2000000 Filler ... +2000001 2000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 2000000; +id estRows task access object operator info +TableReader 9.00 root partition:p0,p1 data:Selection +└─Selection 9.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 2000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2000000; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 2000000; +id estRows task access object operator info +TableReader 4.00 root partition:p2 data:Selection +└─Selection 4.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2000000; +a b +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 2000000; +id estRows task access object operator info +TableReader 10.00 root partition:all data:Selection +└─Selection 10.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 2000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2000000; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 2000000; +id estRows task access object operator info +TableReader 5.00 root partition:p2 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2000000; +a b +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 2000000 AND a <= 2000002; +id estRows task access object operator info +TableReader 3.00 root partition:p2 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2000000), le(executor__partition__partition_boundaries.t.a, 2000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2000000 AND a <= 2000002; +a b +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 2000000 AND a <= 2000002; +id estRows task access object operator info +TableReader 2.00 root partition:p2 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2000000), le(executor__partition__partition_boundaries.t.a, 2000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2000000 AND a <= 2000002; +a b +2000001 2000001 Filler ... +2000002 2000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 2000000 AND a < 2000002; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2000000), lt(executor__partition__partition_boundaries.t.a, 2000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2000000 AND a < 2000002; +a b +2000001 2000001 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 2000000 AND a <= 2000002; +id estRows task access object operator info +TableReader 2.00 root partition:p2 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2000000), le(executor__partition__partition_boundaries.t.a, 2000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2000000 AND a <= 2000002; +a b +2000001 2000001 Filler ... +2000002 2000002 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 2999997; +id estRows task access object operator info +TableReader 12.00 root partition:all data:Selection +└─Selection 12.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 2999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2999997; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 2999997; +id estRows task access object operator info +TableReader 2.00 root partition:p2 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2999997; +a b +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 2999997; +id estRows task access object operator info +TableReader 12.00 root partition:all data:Selection +└─Selection 12.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 2999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2999997; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 2999997; +id estRows task access object operator info +TableReader 2.00 root partition:p2 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2999997) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2999997; +a b +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 2999997 AND a <= 2999999; +id estRows task access object operator info +TableReader 2.00 root partition:p2 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2999997), le(executor__partition__partition_boundaries.t.a, 2999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2999997 AND a <= 2999999; +a b +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 2999997 AND a <= 2999999; +id estRows task access object operator info +TableReader 2.00 root partition:p2 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2999997), le(executor__partition__partition_boundaries.t.a, 2999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2999997 AND a <= 2999999; +a b +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 2999997 AND a < 2999999; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2999997), lt(executor__partition__partition_boundaries.t.a, 2999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2999997 AND a < 2999999; +a b +2999998 2999998 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 2999997 AND a <= 2999999; +id estRows task access object operator info +TableReader 2.00 root partition:p2 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2999997), le(executor__partition__partition_boundaries.t.a, 2999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2999997 AND a <= 2999999; +a b +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 2999998; +id estRows task access object operator info +TableReader 12.00 root partition:all data:Selection +└─Selection 12.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 2999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2999998; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 2999998; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2999998; +a b +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a <= 2999998; +id estRows task access object operator info +TableReader 13.00 root partition:all data:Selection +└─Selection 13.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 2999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2999998; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 2999998; +id estRows task access object operator info +TableReader 2.00 root partition:p2 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2999998) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2999998; +a b +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 2999998 AND a <= 3000000; +id estRows task access object operator info +TableReader 2.00 root partition:p2 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2999998), le(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2999998 AND a <= 3000000; +a b +2999998 2999998 Filler ... +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 2999998 AND a <= 3000000; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2999998), le(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2999998 AND a <= 3000000; +a b +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 2999998 AND a < 3000000; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2999998), lt(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2999998 AND a < 3000000; +a b +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 2999998 AND a <= 3000000; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2999998), le(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2999998 AND a <= 3000000; +a b +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a < 2999999; +id estRows task access object operator info +TableReader 13.00 root partition:all data:Selection +└─Selection 13.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 2999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2999999; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 2999999; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2999999; +a b +explain format='brief' SELECT * FROM t WHERE a <= 2999999; +id estRows task access object operator info +TableReader 14.00 root partition:all data:Selection +└─Selection 14.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 2999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2999999; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 2999999; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2999999) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2999999; +a b +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 2999999 AND a <= 3000001; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2999999), le(executor__partition__partition_boundaries.t.a, 3000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2999999 AND a <= 3000001; +a b +2999999 2999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 2999999 AND a <= 3000001; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2999999), le(executor__partition__partition_boundaries.t.a, 3000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2999999 AND a <= 3000001; +a b +explain format='brief' SELECT * FROM t WHERE a > 2999999 AND a < 3000001; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2999999), lt(executor__partition__partition_boundaries.t.a, 3000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2999999 AND a < 3000001; +a b +explain format='brief' SELECT * FROM t WHERE a > 2999999 AND a <= 3000001; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2999999), le(executor__partition__partition_boundaries.t.a, 3000001) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2999999 AND a <= 3000001; +a b +explain format='brief' SELECT * FROM t WHERE a < 3000000; +id estRows task access object operator info +TableReader 14.00 root partition:all data:Selection +└─Selection 14.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 3000000; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a > 3000000; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 3000000; +a b +explain format='brief' SELECT * FROM t WHERE a <= 3000000; +id estRows task access object operator info +TableReader 14.00 root partition:all data:Selection +└─Selection 14.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 3000000; +a b +-2147483648 MIN_INT filler... +0 0 Filler... +1000000 1000000 Filler ... +1000001 1000001 Filler ... +1000002 1000002 Filler ... +1999998 1999998 Filler ... +1999999 1999999 Filler ... +2000000 2000000 Filler ... +2000001 2000001 Filler ... +2000002 2000002 Filler ... +2999998 2999998 Filler ... +2999999 2999999 Filler ... +999998 999998 Filler ... +999999 999999 Filler ... +explain format='brief' SELECT * FROM t WHERE a >= 3000000; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000000) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 3000000; +a b +explain format='brief' SELECT * FROM t WHERE a >= 3000000 AND a <= 3000002; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3000000), le(executor__partition__partition_boundaries.t.a, 3000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 3000000 AND a <= 3000002; +a b +explain format='brief' SELECT * FROM t WHERE a > 3000000 AND a <= 3000002; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 3000000), le(executor__partition__partition_boundaries.t.a, 3000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 3000000 AND a <= 3000002; +a b +explain format='brief' SELECT * FROM t WHERE a > 3000000 AND a < 3000002; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 3000000), lt(executor__partition__partition_boundaries.t.a, 3000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 3000000 AND a < 3000002; +a b +explain format='brief' SELECT * FROM t WHERE a > 3000000 AND a <= 3000002; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 3000000), le(executor__partition__partition_boundaries.t.a, 3000002) + └─TableFullScan 14.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 3000000 AND a <= 3000002; +a b +set @@tidb_partition_prune_mode = default; +set @@tidb_partition_prune_mode = 'dynamic'; +drop table if exists t; +CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( +PARTITION p0 VALUES LESS THAN (1), +PARTITION p1 VALUES LESS THAN (2), +PARTITION p2 VALUES LESS THAN (3), +PARTITION p3 VALUES LESS THAN (4), +PARTITION p4 VALUES LESS THAN (5), +PARTITION p5 VALUES LESS THAN (6), +PARTITION p6 VALUES LESS THAN (7)); +INSERT INTO t VALUES (0, '0 Filler...'); +INSERT INTO t VALUES (1, '1 Filler...'); +INSERT INTO t VALUES (2, '2 Filler...'); +INSERT INTO t VALUES (3, '3 Filler...'); +INSERT INTO t VALUES (4, '4 Filler...'); +INSERT INTO t VALUES (5, '5 Filler...'); +INSERT INTO t VALUES (6, '6 Filler...'); +ANALYZE TABLE t all columns; +explain format='brief' SELECT * FROM t WHERE a < -1; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, -1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < -1; +a b +explain format='brief' SELECT * FROM t WHERE a > -1; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, -1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > -1; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= -1; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, -1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= -1; +a b +explain format='brief' SELECT * FROM t WHERE a >= -1; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, -1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= -1; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a > -1; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, -1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a > -1; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a < -1; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, -1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a < -1; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a > -1); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, -1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a > -1); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a < -1); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, -1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a < -1); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a >= -1; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, -1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a >= -1; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a < -1; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, -1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a < -1; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a >= -1); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, -1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a >= -1); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a < -1); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, -1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a < -1); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a > -1; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, -1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a > -1; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a <= -1; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, -1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a <= -1; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a > -1); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, -1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a > -1); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a <= -1); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, -1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a <= -1); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a >= -1; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, -1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a >= -1; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a <= -1; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, -1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a <= -1; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a >= -1); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, -1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a >= -1); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a <= -1); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, -1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a <= -1); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 0; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 0; +a b +explain format='brief' SELECT * FROM t WHERE a > 0; +id estRows task access object operator info +TableReader 6.00 root partition:p1,p2,p3,p4,p5,p6 data:Selection +└─Selection 6.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 0; +a b +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 0; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 0; +a b +0 0 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 0; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 0; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a > 0; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 0)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a > 0; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a < 0; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a < 0; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a > 0); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 0)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a > 0); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a < 0); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 0)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a < 0); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a >= 0; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 0)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a >= 0; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a < 0; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a < 0; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a >= 0); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 0)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a >= 0); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a < 0); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 0)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a < 0); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a > 0; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 0)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a > 0; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a <= 0; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a <= 0; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a > 0); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 0)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a > 0); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a <= 0); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 0)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a <= 0); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a >= 0; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 0)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a >= 0; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a <= 0; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 0) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a <= 0; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a >= 0); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 0)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a >= 0); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a <= 0); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 0)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a <= 0); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 1; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 1; +a b +0 0 Filler... +explain format='brief' SELECT * FROM t WHERE a > 1; +id estRows task access object operator info +TableReader 5.00 root partition:p2,p3,p4,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 1; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 1; +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 1; +a b +0 0 Filler... +1 1 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 1; +id estRows task access object operator info +TableReader 6.00 root partition:p1,p2,p3,p4,p5,p6 data:Selection +└─Selection 6.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 1; +a b +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a > 1; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a > 1; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a < 1; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a < 1; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a > 1); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a > 1); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a < 1); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a < 1); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a >= 1; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a >= 1; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a < 1; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a < 1; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a >= 1); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a >= 1); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a < 1); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a < 1); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a > 1; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a > 1; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a <= 1; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a <= 1; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a > 1); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a > 1); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a <= 1); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a <= 1); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a >= 1; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a >= 1; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a <= 1; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 1) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a <= 1; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a >= 1); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a >= 1); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a <= 1); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 1)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a <= 1); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2; +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 2) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2; +a b +0 0 Filler... +1 1 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2; +id estRows task access object operator info +TableReader 4.00 root partition:p3,p4,p5,p6 data:Selection +└─Selection 4.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2; +a b +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2; +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1,p2 data:Selection +└─Selection 3.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 2) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2; +id estRows task access object operator info +TableReader 5.00 root partition:p2,p3,p4,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a > 2; +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1,p3,p4,p5,p6 data:Selection +└─Selection 6.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 2)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a > 2; +a b +0 0 Filler... +1 1 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a < 2; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 2) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a < 2; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a > 2); +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 2)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a > 2); +a b +2 2 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a < 2); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 2)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a < 2); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a >= 2; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 2)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a >= 2; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a < 2; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 2) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a < 2; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a >= 2); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 2)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a >= 2); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a < 2); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 2)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a < 2); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a > 2; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 2)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a > 2; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a <= 2; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 2) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a <= 2; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a > 2); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 2)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a > 2); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a <= 2); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 2)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a <= 2); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a >= 2; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 2)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a >= 2; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a <= 2; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 2) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a <= 2; +a b +2 2 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a >= 2); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 2)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a >= 2); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a <= 2); +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1,p3,p4,p5,p6 data:Selection +└─Selection 6.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 2)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a <= 2); +a b +0 0 Filler... +1 1 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 3; +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1,p2 data:Selection +└─Selection 3.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 3) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 3; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +explain format='brief' SELECT * FROM t WHERE a > 3; +id estRows task access object operator info +TableReader 3.00 root partition:p4,p5,p6 data:Selection +└─Selection 3.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 3) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 3; +a b +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 3; +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1,p2,p3 data:Selection +└─Selection 4.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 3) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 3; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 3; +id estRows task access object operator info +TableReader 4.00 root partition:p3,p4,p5,p6 data:Selection +└─Selection 4.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 3) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 3; +a b +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a > 3; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1,p4,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 3)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a > 3; +a b +0 0 Filler... +1 1 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a < 3; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 3) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a < 3; +a b +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a > 3); +id estRows task access object operator info +TableReader 2.00 root partition:p2,p3 data:Selection +└─Selection 2.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 3)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a > 3); +a b +2 2 Filler... +3 3 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a < 3); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 3)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a < 3); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a >= 3; +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1,p3,p4,p5,p6 data:Selection +└─Selection 6.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 3)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a >= 3; +a b +0 0 Filler... +1 1 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a < 3; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 3) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a < 3; +a b +2 2 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a >= 3); +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 3)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a >= 3); +a b +2 2 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a < 3); +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1,p3,p4,p5,p6 data:Selection +└─Selection 6.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 3)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a < 3); +a b +0 0 Filler... +1 1 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a > 3; +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1,p2,p4,p5,p6 data:Selection +└─Selection 6.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 3)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a > 3; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a <= 3; +id estRows task access object operator info +TableReader 1.00 root partition:p3 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 3) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a <= 3; +a b +3 3 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a > 3); +id estRows task access object operator info +TableReader 1.00 root partition:p3 data:Selection +└─Selection 1.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 3)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a > 3); +a b +3 3 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a <= 3); +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1,p2,p4,p5,p6 data:Selection +└─Selection 6.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 3)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a <= 3); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a >= 3; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 3)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a >= 3; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a <= 3; +id estRows task access object operator info +TableReader 2.00 root partition:p2,p3 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 3) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a <= 3; +a b +2 2 Filler... +3 3 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a >= 3); +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 3)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a >= 3); +a b +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a <= 3); +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1,p4,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 3)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a <= 3); +a b +0 0 Filler... +1 1 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 4; +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1,p2,p3 data:Selection +└─Selection 4.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 4; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +explain format='brief' SELECT * FROM t WHERE a > 4; +id estRows task access object operator info +TableReader 2.00 root partition:p5,p6 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 4; +a b +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 4; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1,p2,p3,p4 data:Selection +└─Selection 5.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 4; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 4; +id estRows task access object operator info +TableReader 3.00 root partition:p4,p5,p6 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 4; +a b +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a > 4; +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1,p5,p6 data:Selection +└─Selection 4.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 4)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a > 4; +a b +0 0 Filler... +1 1 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a < 4; +id estRows task access object operator info +TableReader 1.00 root partition:p3 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a < 4; +a b +3 3 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a > 4); +id estRows task access object operator info +TableReader 3.00 root partition:p2,p3,p4 data:Selection +└─Selection 3.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 4)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a > 4); +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a < 4); +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1,p2,p4,p5,p6 data:Selection +└─Selection 6.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 4)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a < 4); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a >= 4; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1,p4,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 4)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a >= 4; +a b +0 0 Filler... +1 1 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a < 4; +id estRows task access object operator info +TableReader 2.00 root partition:p2,p3 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a < 4; +a b +2 2 Filler... +3 3 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a >= 4); +id estRows task access object operator info +TableReader 2.00 root partition:p2,p3 data:Selection +└─Selection 2.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 4)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a >= 4); +a b +2 2 Filler... +3 3 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a < 4); +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1,p4,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 4)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a < 4); +a b +0 0 Filler... +1 1 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a > 4; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1,p2,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 4)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a > 4; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a <= 4; +id estRows task access object operator info +TableReader 2.00 root partition:p3,p4 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a <= 4; +a b +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a > 4); +id estRows task access object operator info +TableReader 2.00 root partition:p3,p4 data:Selection +└─Selection 2.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 4)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a > 4); +a b +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a <= 4); +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1,p2,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 4)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a <= 4); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a >= 4; +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1,p2,p4,p5,p6 data:Selection +└─Selection 6.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 4)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a >= 4; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a <= 4; +id estRows task access object operator info +TableReader 3.00 root partition:p2,p3,p4 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 4) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a <= 4; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a >= 4); +id estRows task access object operator info +TableReader 1.00 root partition:p3 data:Selection +└─Selection 1.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 4)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a >= 4); +a b +3 3 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a <= 4); +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1,p5,p6 data:Selection +└─Selection 4.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 4)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a <= 4); +a b +0 0 Filler... +1 1 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 5; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1,p2,p3,p4 data:Selection +└─Selection 5.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 5) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 5; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE a > 5; +id estRows task access object operator info +TableReader 1.00 root partition:p6 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 5) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 5; +a b +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 5; +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1,p2,p3,p4,p5 data:Selection +└─Selection 6.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 5) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 5; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 5; +id estRows task access object operator info +TableReader 2.00 root partition:p5,p6 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 5) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 5; +a b +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a > 5; +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1,p6 data:Selection +└─Selection 3.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 5)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a > 5; +a b +0 0 Filler... +1 1 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a < 5; +id estRows task access object operator info +TableReader 2.00 root partition:p3,p4 data:Selection +└─Selection 2.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 5) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a < 5; +a b +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a > 5); +id estRows task access object operator info +TableReader 4.00 root partition:p2,p3,p4,p5 data:Selection +└─Selection 4.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 5)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a > 5); +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a < 5); +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1,p2,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 5)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a < 5); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a >= 5; +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1,p5,p6 data:Selection +└─Selection 4.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 5)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a >= 5; +a b +0 0 Filler... +1 1 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a < 5; +id estRows task access object operator info +TableReader 3.00 root partition:p2,p3,p4 data:Selection +└─Selection 3.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 5) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a < 5; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a >= 5); +id estRows task access object operator info +TableReader 3.00 root partition:p2,p3,p4 data:Selection +└─Selection 3.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 5)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a >= 5); +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a < 5); +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1,p5,p6 data:Selection +└─Selection 4.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 5)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a < 5); +a b +0 0 Filler... +1 1 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a > 5; +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1,p2,p6 data:Selection +└─Selection 4.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 5)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a > 5; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a <= 5; +id estRows task access object operator info +TableReader 3.00 root partition:p3,p4,p5 data:Selection +└─Selection 3.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 5) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a <= 5; +a b +3 3 Filler... +4 4 Filler... +5 5 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a > 5); +id estRows task access object operator info +TableReader 3.00 root partition:p3,p4,p5 data:Selection +└─Selection 3.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 5)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a > 5); +a b +3 3 Filler... +4 4 Filler... +5 5 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a <= 5); +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1,p2,p6 data:Selection +└─Selection 4.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 5)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a <= 5); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a >= 5; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1,p2,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 5)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a >= 5; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a <= 5; +id estRows task access object operator info +TableReader 4.00 root partition:p2,p3,p4,p5 data:Selection +└─Selection 4.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 5) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a <= 5; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a >= 5); +id estRows task access object operator info +TableReader 2.00 root partition:p3,p4 data:Selection +└─Selection 2.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 5)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a >= 5); +a b +3 3 Filler... +4 4 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a <= 5); +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1,p6 data:Selection +└─Selection 3.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 5)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a <= 5); +a b +0 0 Filler... +1 1 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 6; +id estRows task access object operator info +TableReader 6.00 root partition:p0,p1,p2,p3,p4,p5 data:Selection +└─Selection 6.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 6) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 6; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +explain format='brief' SELECT * FROM t WHERE a > 6; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 6) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 6; +a b +explain format='brief' SELECT * FROM t WHERE a <= 6; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 6) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 6; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 6; +id estRows task access object operator info +TableReader 1.00 root partition:p6 data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 6) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 6; +a b +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a > 6; +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 6)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a > 6; +a b +0 0 Filler... +1 1 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a < 6; +id estRows task access object operator info +TableReader 3.00 root partition:p3,p4,p5 data:Selection +└─Selection 3.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 6) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a < 6; +a b +3 3 Filler... +4 4 Filler... +5 5 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a > 6); +id estRows task access object operator info +TableReader 5.00 root partition:p2,p3,p4,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 6)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a > 6); +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a < 6); +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1,p2,p6 data:Selection +└─Selection 4.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 6)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a < 6); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a >= 6; +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1,p6 data:Selection +└─Selection 3.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 6)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a >= 6; +a b +0 0 Filler... +1 1 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a < 6; +id estRows task access object operator info +TableReader 4.00 root partition:p2,p3,p4,p5 data:Selection +└─Selection 4.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 6) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a < 6; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a >= 6); +id estRows task access object operator info +TableReader 4.00 root partition:p2,p3,p4,p5 data:Selection +└─Selection 4.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 6)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a >= 6); +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a < 6); +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1,p6 data:Selection +└─Selection 3.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 6)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a < 6); +a b +0 0 Filler... +1 1 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a > 6; +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1,p2 data:Selection +└─Selection 3.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 6)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a > 6; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a <= 6; +id estRows task access object operator info +TableReader 4.00 root partition:p3,p4,p5,p6 data:Selection +└─Selection 4.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 6) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a <= 6; +a b +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a > 6); +id estRows task access object operator info +TableReader 4.00 root partition:p3,p4,p5,p6 data:Selection +└─Selection 4.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 6)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a > 6); +a b +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a <= 6); +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1,p2 data:Selection +└─Selection 3.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 6)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a <= 6); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a >= 6; +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1,p2,p6 data:Selection +└─Selection 4.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 6)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a >= 6; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a <= 6; +id estRows task access object operator info +TableReader 5.00 root partition:p2,p3,p4,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 6) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a <= 6; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a >= 6); +id estRows task access object operator info +TableReader 3.00 root partition:p3,p4,p5 data:Selection +└─Selection 3.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 6)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a >= 6); +a b +3 3 Filler... +4 4 Filler... +5 5 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a <= 6); +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 6)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a <= 6); +a b +0 0 Filler... +1 1 Filler... +explain format='brief' SELECT * FROM t WHERE a < 7; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] lt(executor__partition__partition_boundaries.t.a, 7) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 7; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a > 7; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 7) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 7; +a b +explain format='brief' SELECT * FROM t WHERE a <= 7; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] le(executor__partition__partition_boundaries.t.a, 7) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 7; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 7; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 7) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 7; +a b +explain format='brief' SELECT * FROM t WHERE a < 2 OR a > 7; +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 7)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a > 7; +a b +0 0 Filler... +1 1 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a < 7; +id estRows task access object operator info +TableReader 4.00 root partition:p3,p4,p5,p6 data:Selection +└─Selection 4.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 7) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a < 7; +a b +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a > 7); +id estRows task access object operator info +TableReader 5.00 root partition:p2,p3,p4,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 7)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a > 7); +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a < 7); +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1,p2 data:Selection +└─Selection 3.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 7)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a < 7); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +explain format='brief' SELECT * FROM t WHERE a < 2 OR a >= 7; +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 7)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a < 2 OR a >= 7; +a b +0 0 Filler... +1 1 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a < 7; +id estRows task access object operator info +TableReader 5.00 root partition:p2,p3,p4,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 7) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a < 7; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a < 2 OR a >= 7); +id estRows task access object operator info +TableReader 5.00 root partition:p2,p3,p4,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] and(ge(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 7)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a < 2 OR a >= 7); +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a < 7); +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 7)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a < 7); +a b +0 0 Filler... +1 1 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a > 7; +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1,p2 data:Selection +└─Selection 3.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 7)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a > 7; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +explain format='brief' SELECT * FROM t WHERE a > 2 AND a <= 7; +id estRows task access object operator info +TableReader 4.00 root partition:p3,p4,p5,p6 data:Selection +└─Selection 4.00 cop[tikv] gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 7) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a > 2 AND a <= 7; +a b +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a > 7); +id estRows task access object operator info +TableReader 4.00 root partition:p3,p4,p5,p6 data:Selection +└─Selection 4.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 7)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a > 7); +a b +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a > 2 AND a <= 7); +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1,p2 data:Selection +└─Selection 3.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 7)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a > 2 AND a <= 7); +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +explain format='brief' SELECT * FROM t WHERE a <= 2 OR a >= 7; +id estRows task access object operator info +TableReader 3.00 root partition:p0,p1,p2 data:Selection +└─Selection 3.00 cop[tikv] or(le(executor__partition__partition_boundaries.t.a, 2), ge(executor__partition__partition_boundaries.t.a, 7)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a <= 2 OR a >= 7; +a b +0 0 Filler... +1 1 Filler... +2 2 Filler... +explain format='brief' SELECT * FROM t WHERE a >= 2 AND a <= 7; +id estRows task access object operator info +TableReader 5.00 root partition:p2,p3,p4,p5,p6 data:Selection +└─Selection 5.00 cop[tikv] ge(executor__partition__partition_boundaries.t.a, 2), le(executor__partition__partition_boundaries.t.a, 7) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE a >= 2 AND a <= 7; +a b +2 2 Filler... +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a <= 2 OR a >= 7); +id estRows task access object operator info +TableReader 4.00 root partition:p3,p4,p5,p6 data:Selection +└─Selection 4.00 cop[tikv] and(gt(executor__partition__partition_boundaries.t.a, 2), lt(executor__partition__partition_boundaries.t.a, 7)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a <= 2 OR a >= 7); +a b +3 3 Filler... +4 4 Filler... +5 5 Filler... +6 6 Filler... +explain format='brief' SELECT * FROM t WHERE NOT (a >= 2 AND a <= 7); +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] or(lt(executor__partition__partition_boundaries.t.a, 2), gt(executor__partition__partition_boundaries.t.a, 7)) + └─TableFullScan 7.00 cop[tikv] table:t keep order:false +SELECT * FROM t WHERE NOT (a >= 2 AND a <= 7); +a b +0 0 Filler... +1 1 Filler... +set @@tidb_partition_prune_mode = default; diff --git a/tests/integrationtest/r/executor/partition/partition_with_expression.result b/tests/integrationtest/r/executor/partition/partition_with_expression.result new file mode 100644 index 0000000000000..81c168a8d0bc1 --- /dev/null +++ b/tests/integrationtest/r/executor/partition/partition_with_expression.result @@ -0,0 +1,1250 @@ +drop table if exists tp, t; +set tidb_partition_prune_mode='dynamic'; +create table tp(a datetime, b int) partition by range columns (a) (partition p0 values less than("2012-12-10 00:00:00"), partition p1 values less than("2022-12-30 00:00:00"), partition p2 values less than("2025-12-12 00:00:00")); +create table t(a datetime, b int) partition by range columns (a) (partition p0 values less than("2012-12-10 00:00:00"), partition p1 values less than("2022-12-30 00:00:00"), partition p2 values less than("2025-12-12 00:00:00")); +insert into tp values("2015-09-09 00:00:00", 1), ("2020-08-08 19:00:01", 2), ("2024-01-01 01:01:01", 3); +insert into t values("2015-09-09 00:00:00", 1), ("2020-08-08 19:00:01", 2), ("2024-01-01 01:01:01", 3); +analyze table tp all columns; +analyze table t all columns; +explain format='brief' select * from tp where a != '2024-01-01 01:01:01'; +id estRows task access object operator info +TableReader 2.00 root partition:all data:Selection +└─Selection 2.00 cop[tikv] ne(executor__partition__partition_with_expression.tp.a, 2024-01-01 01:01:01.000000) + └─TableFullScan 3.00 cop[tikv] table:tp keep order:false +select * from tp where a != '2024-01-01 01:01:01'; +a b +2015-09-09 00:00:00 1 +2020-08-08 19:00:01 2 +select * from t where a != '2024-01-01 01:01:01'; +a b +2015-09-09 00:00:00 1 +2020-08-08 19:00:01 2 +explain format='brief' select * from tp where a != '2024-01-01 01:01:01' and a > '2015-09-09 00:00:00'; +id estRows task access object operator info +TableReader 1.00 root partition:p1,p2 data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_with_expression.tp.a, 2015-09-09 00:00:00.000000), ne(executor__partition__partition_with_expression.tp.a, 2024-01-01 01:01:01.000000) + └─TableFullScan 3.00 cop[tikv] table:tp keep order:false +select * from tp where a != '2024-01-01 01:01:01' and a > '2015-09-09 00:00:00'; +a b +2020-08-08 19:00:01 2 +select * from t where a != '2024-01-01 01:01:01' and a > '2015-09-09 00:00:00'; +a b +2020-08-08 19:00:01 2 +set tidb_partition_prune_mode=default; +drop table if exists tp, t; +set tidb_partition_prune_mode='dynamic'; +create table tp(a datetime, b int) partition by range(weekday(a)) (partition p0 values less than(3), partition p1 values less than(5), partition p2 values less than(8)); +create table t(a datetime, b int); +insert into tp values("2020-08-17 00:00:00", 1), ("2020-08-18 00:00:00", 2), ("2020-08-19 00:00:00", 4), ("2020-08-20 00:00:00", 5), ("2020-08-21 00:00:00", 6), ("2020-08-22 00:00:00", 0); +insert into t values("2020-08-17 00:00:00", 1), ("2020-08-18 00:00:00", 2), ("2020-08-19 00:00:00", 4), ("2020-08-20 00:00:00", 5), ("2020-08-21 00:00:00", 6), ("2020-08-22 00:00:00", 0); +analyze table tp all columns; +analyze table t all columns; +explain format='brief' select * from tp where a = '2020-08-17 00:00:00'; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_with_expression.tp.a, 2020-08-17 00:00:00.000000) + └─TableFullScan 6.00 cop[tikv] table:tp keep order:false +select * from tp where a = '2020-08-17 00:00:00'; +a b +2020-08-17 00:00:00 1 +select * from t where a = '2020-08-17 00:00:00'; +a b +2020-08-17 00:00:00 1 +explain format='brief' select * from tp where a= '2020-08-20 00:00:00' and a < '2020-08-22 00:00:00'; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_with_expression.tp.a, 2020-08-20 00:00:00.000000) + └─TableFullScan 6.00 cop[tikv] table:tp keep order:false +select * from tp where a= '2020-08-20 00:00:00' and a < '2020-08-22 00:00:00'; +a b +2020-08-20 00:00:00 5 +select * from t where a= '2020-08-20 00:00:00' and a < '2020-08-22 00:00:00'; +a b +2020-08-20 00:00:00 5 +explain format='brief' select * from tp where a < '2020-08-19 00:00:00'; +id estRows task access object operator info +TableReader 2.00 root partition:all data:Selection +└─Selection 2.00 cop[tikv] lt(executor__partition__partition_with_expression.tp.a, 2020-08-19 00:00:00.000000) + └─TableFullScan 6.00 cop[tikv] table:tp keep order:false +select * from tp where a < '2020-08-19 00:00:00'; +a b +2020-08-17 00:00:00 1 +2020-08-18 00:00:00 2 +select * from t where a < '2020-08-19 00:00:00'; +a b +2020-08-17 00:00:00 1 +2020-08-18 00:00:00 2 +set tidb_partition_prune_mode=default; +drop table if exists tp, t; +set tidb_partition_prune_mode='dynamic'; +create table tp(a timestamp, b int) partition by range(floor(unix_timestamp(a))) (partition p0 values less than(1580670000), partition p1 values less than(1597622400), partition p2 values less than(1629158400)); +create table t(a timestamp, b int); +insert into tp values('2020-01-01 19:00:00', 1),('2020-08-15 00:00:00', -1), ('2020-08-18 05:00:01', 2), ('2020-10-01 14:13:15', 3); +insert into t values('2020-01-01 19:00:00', 1),('2020-08-15 00:00:00', -1), ('2020-08-18 05:00:01', 2), ('2020-10-01 14:13:15', 3); +analyze table tp all columns; +analyze table t all columns; +explain select * from tp where a > '2020-09-11 00:00:00'; +id estRows task access object operator info +TableReader_7 1.00 root partition:p2 data:Selection_6 +└─Selection_6 1.00 cop[tikv] gt(executor__partition__partition_with_expression.tp.a, 2020-09-11 00:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tp keep order:false +select * from tp where a > '2020-09-11 00:00:00'; +a b +2020-10-01 14:13:15 3 +select * from t where a > '2020-09-11 00:00:00'; +a b +2020-10-01 14:13:15 3 +explain select * from tp where a < '2020-07-07 01:00:00'; +id estRows task access object operator info +TableReader_7 1.00 root partition:p0,p1 data:Selection_6 +└─Selection_6 1.00 cop[tikv] lt(executor__partition__partition_with_expression.tp.a, 2020-07-07 01:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tp keep order:false +select * from tp where a < '2020-07-07 01:00:00'; +a b +2020-01-01 19:00:00 1 +select * from t where a < '2020-07-07 01:00:00'; +a b +2020-01-01 19:00:00 1 +set tidb_partition_prune_mode=default; +drop table if exists tp, t; +set tidb_partition_prune_mode='dynamic'; +create table tp(a timestamp, b int) partition by range(unix_timestamp(a)) (partition p0 values less than(1580670000), partition p1 values less than(1597622400), partition p2 values less than(1629158400)); +create table t(a timestamp, b int); +insert into tp values('2020-01-01 19:00:00', 1),('2020-08-15 00:00:00', -1), ('2020-08-18 05:00:01', 2), ('2020-10-01 14:13:15', 3); +insert into t values('2020-01-01 19:00:00', 1),('2020-08-15 00:00:00', -1), ('2020-08-18 05:00:01', 2), ('2020-10-01 14:13:15', 3); +analyze table tp all columns; +analyze table t all columns; +explain select * from tp where a > '2020-09-11 00:00:00'; +id estRows task access object operator info +TableReader_7 1.00 root partition:p2 data:Selection_6 +└─Selection_6 1.00 cop[tikv] gt(executor__partition__partition_with_expression.tp.a, 2020-09-11 00:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tp keep order:false +select * from tp where a > '2020-09-11 00:00:00'; +a b +2020-10-01 14:13:15 3 +select * from t where a > '2020-09-11 00:00:00'; +a b +2020-10-01 14:13:15 3 +explain select * from tp where a < '2020-07-07 01:00:00'; +id estRows task access object operator info +TableReader_7 1.00 root partition:p0,p1 data:Selection_6 +└─Selection_6 1.00 cop[tikv] lt(executor__partition__partition_with_expression.tp.a, 2020-07-07 01:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tp keep order:false +select * from tp where a < '2020-07-07 01:00:00'; +a b +2020-01-01 19:00:00 1 +select * from t where a < '2020-07-07 01:00:00'; +a b +2020-01-01 19:00:00 1 +set tidb_partition_prune_mode=default; +drop table if exists tp, t; +set tidb_partition_prune_mode='dynamic'; +create table tp(a datetime, b int) partition by range columns(a) (partition p0 values less than('2020-02-02 00:00:00'), partition p1 values less than('2020-09-01 00:00:00'), partition p2 values less than('2020-12-20 00:00:00')); +create table t(a datetime, b int); +insert into tp values('2020-01-01 12:00:00', 1), ('2020-08-22 10:00:00', 2), ('2020-09-09 11:00:00', 3), ('2020-10-01 00:00:00', 4); +insert into t values('2020-01-01 12:00:00', 1), ('2020-08-22 10:00:00', 2), ('2020-09-09 11:00:00', 3), ('2020-10-01 00:00:00', 4); +analyze table tp all columns; +analyze table t all columns; +explain select * from tp where a < '2020-09-01 00:00:00'; +id estRows task access object operator info +TableReader_7 2.00 root partition:p0,p1 data:Selection_6 +└─Selection_6 2.00 cop[tikv] lt(executor__partition__partition_with_expression.tp.a, 2020-09-01 00:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tp keep order:false +select * from tp where a < '2020-09-01 00:00:00'; +a b +2020-01-01 12:00:00 1 +2020-08-22 10:00:00 2 +select * from t where a < '2020-09-01 00:00:00'; +a b +2020-01-01 12:00:00 1 +2020-08-22 10:00:00 2 +explain select * from tp where a > '2020-07-07 01:00:00'; +id estRows task access object operator info +TableReader_7 3.00 root partition:p1,p2 data:Selection_6 +└─Selection_6 3.00 cop[tikv] gt(executor__partition__partition_with_expression.tp.a, 2020-07-07 01:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tp keep order:false +select * from tp where a > '2020-07-07 01:00:00'; +a b +2020-08-22 10:00:00 2 +2020-09-09 11:00:00 3 +2020-10-01 00:00:00 4 +select * from t where a > '2020-07-07 01:00:00'; +a b +2020-08-22 10:00:00 2 +2020-09-09 11:00:00 3 +2020-10-01 00:00:00 4 +set tidb_partition_prune_mode=default; +drop table if exists tp, t; +set tidb_partition_prune_mode='dynamic'; +create table tp(a varchar(255), b int) partition by range columns(a) (partition p0 values less than('ddd'), partition p1 values less than('ggggg'), partition p2 values less than('mmmmmm')); +create table t(a varchar(255), b int); +insert into tp values('aaa', 1), ('bbbb', 2), ('ccc', 3), ('dfg', 4), ('kkkk', 5), ('10', 6); +insert into t values('aaa', 1), ('bbbb', 2), ('ccc', 3), ('dfg', 4), ('kkkk', 5), ('10', 6); +analyze table tp all columns; +analyze table t all columns; +explain select * from tp where a < '10'; +id estRows task access object operator info +TableReader_7 1.00 root partition:p0 data:Selection_6 +└─Selection_6 1.00 cop[tikv] lt(executor__partition__partition_with_expression.tp.a, "10") + └─TableFullScan_5 6.00 cop[tikv] table:tp keep order:false +select * from tp where a < '10'; +a b +select * from t where a < '10'; +a b +explain select * from tp where a > 0; +id estRows task access object operator info +TableReader_7 4.80 root partition:all data:Selection_6 +└─Selection_6 4.80 cop[tikv] gt(cast(executor__partition__partition_with_expression.tp.a, double BINARY), 0) + └─TableFullScan_5 6.00 cop[tikv] table:tp keep order:false +select * from tp where a > 0; +a b +10 6 +select * from t where a > 0; +a b +10 6 +explain select * from tp where a < 0; +id estRows task access object operator info +TableReader_7 4.80 root partition:all data:Selection_6 +└─Selection_6 4.80 cop[tikv] lt(cast(executor__partition__partition_with_expression.tp.a, double BINARY), 0) + └─TableFullScan_5 6.00 cop[tikv] table:tp keep order:false +select * from tp where a < 0; +a b +select * from t where a < 0; +a b +set tidb_partition_prune_mode=default; +drop table if exists trange, thash, t; +create table trange(a int, b int) partition by range(a) (partition p0 values less than(3), partition p1 values less than (5), partition p2 values less than(11)); +create table thash(a int, b int) partition by hash(a) partitions 4; +create table t(a int, b int); +insert into trange values(1, NULL), (1, NULL), (1, 1), (2, 1), (3, 2), (4, 3), (5, 5), (6, 7), (7, 7), (7, 7), (10, NULL), (NULL, NULL), (NULL, 1); +insert into thash values(1, NULL), (1, NULL), (1, 1), (2, 1), (3, 2), (4, 3), (5, 5), (6, 7), (7, 7), (7, 7), (10, NULL), (NULL, NULL), (NULL, 1); +insert into t values(1, NULL), (1, NULL), (1, 1), (2, 1), (3, 2), (4, 3), (5, 5), (6, 7), (7, 7), (7, 7), (10, NULL), (NULL, NULL), (NULL, 1); +set session tidb_partition_prune_mode='dynamic'; +analyze table trange all columns; +analyze table thash all columns; +analyze table t all columns; +SELECT * from t where a = 2; +a b +2 1 +explain format='brief' select * from trange where a = 2; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_with_expression.trange.a, 2) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a = 2; +a b +2 1 +explain format='brief' select * from thash where a = 2; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_with_expression.thash.a, 2) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a = 2; +a b +2 1 +SELECT * from t where a = 4 or a = 1; +a b +1 NULL +1 NULL +1 1 +4 3 +explain format='brief' select * from trange where a = 4 or a = 1; +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1 data:Selection +└─Selection 4.00 cop[tikv] or(eq(executor__partition__partition_with_expression.trange.a, 4), eq(executor__partition__partition_with_expression.trange.a, 1)) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a = 4 or a = 1; +a b +1 NULL +1 NULL +1 1 +4 3 +explain format='brief' select * from thash where a = 4 or a = 1; +id estRows task access object operator info +TableReader 4.00 root partition:p0,p1 data:Selection +└─Selection 4.00 cop[tikv] or(eq(executor__partition__partition_with_expression.thash.a, 4), eq(executor__partition__partition_with_expression.thash.a, 1)) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a = 4 or a = 1; +a b +1 NULL +1 NULL +1 1 +4 3 +SELECT * from t where a = -1; +a b +explain format='brief' select * from trange where a = -1; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_with_expression.trange.a, -1) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a = -1; +a b +explain format='brief' select * from thash where a = -1; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_with_expression.thash.a, -1) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a = -1; +a b +SELECT * from t where a is NULL; +a b +NULL NULL +NULL 1 +explain format='brief' select * from trange where a is NULL; +id estRows task access object operator info +TableReader 2.00 root partition:p0 data:Selection +└─Selection 2.00 cop[tikv] isnull(executor__partition__partition_with_expression.trange.a) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a is NULL; +a b +NULL NULL +NULL 1 +explain format='brief' select * from thash where a is NULL; +id estRows task access object operator info +TableReader 2.00 root partition:p0 data:Selection +└─Selection 2.00 cop[tikv] isnull(executor__partition__partition_with_expression.thash.a) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a is NULL; +a b +NULL NULL +NULL 1 +SELECT * from t where b is NULL; +a b +NULL NULL +1 NULL +1 NULL +10 NULL +explain format='brief' select * from trange where b is NULL; +id estRows task access object operator info +TableReader 4.00 root partition:all data:Selection +└─Selection 4.00 cop[tikv] isnull(executor__partition__partition_with_expression.trange.b) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where b is NULL; +a b +NULL NULL +1 NULL +1 NULL +10 NULL +explain format='brief' select * from thash where b is NULL; +id estRows task access object operator info +TableReader 4.00 root partition:all data:Selection +└─Selection 4.00 cop[tikv] isnull(executor__partition__partition_with_expression.thash.b) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where b is NULL; +a b +NULL NULL +1 NULL +1 NULL +10 NULL +SELECT * from t where a > -1; +a b +1 NULL +1 NULL +1 1 +10 NULL +2 1 +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from trange where a > -1; +id estRows task access object operator info +TableReader 11.00 root partition:all data:Selection +└─Selection 11.00 cop[tikv] gt(executor__partition__partition_with_expression.trange.a, -1) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a > -1; +a b +1 NULL +1 NULL +1 1 +10 NULL +2 1 +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from thash where a > -1; +id estRows task access object operator info +TableReader 11.00 root partition:all data:Selection +└─Selection 11.00 cop[tikv] gt(executor__partition__partition_with_expression.thash.a, -1) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a > -1; +a b +1 NULL +1 NULL +1 1 +10 NULL +2 1 +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +SELECT * from t where a >= 4 and a <= 5; +a b +4 3 +5 5 +explain format='brief' select * from trange where a >= 4 and a <= 5; +id estRows task access object operator info +TableReader 2.00 root partition:p1,p2 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_with_expression.trange.a, 4), le(executor__partition__partition_with_expression.trange.a, 5) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a >= 4 and a <= 5; +a b +4 3 +5 5 +explain format='brief' select * from thash where a >= 4 and a <= 5; +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_with_expression.thash.a, 4), le(executor__partition__partition_with_expression.thash.a, 5) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a >= 4 and a <= 5; +a b +4 3 +5 5 +SELECT * from t where a > 10; +a b +explain format='brief' select * from trange where a > 10; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_with_expression.trange.a, 10) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a > 10; +a b +explain format='brief' select * from thash where a > 10; +id estRows task access object operator info +TableReader 1.00 root partition:all data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_with_expression.thash.a, 10) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a > 10; +a b +SELECT * from t where a >=2 and a <= 3; +a b +2 1 +3 2 +explain format='brief' select * from trange where a >=2 and a <= 3; +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_with_expression.trange.a, 2), le(executor__partition__partition_with_expression.trange.a, 3) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a >=2 and a <= 3; +a b +2 1 +3 2 +explain format='brief' select * from thash where a >=2 and a <= 3; +id estRows task access object operator info +TableReader 2.00 root partition:p2,p3 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_with_expression.thash.a, 2), le(executor__partition__partition_with_expression.thash.a, 3) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a >=2 and a <= 3; +a b +2 1 +3 2 +SELECT * from t where a between 2 and 3; +a b +2 1 +3 2 +explain format='brief' select * from trange where a between 2 and 3; +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_with_expression.trange.a, 2), le(executor__partition__partition_with_expression.trange.a, 3) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a between 2 and 3; +a b +2 1 +3 2 +explain format='brief' select * from thash where a between 2 and 3; +id estRows task access object operator info +TableReader 2.00 root partition:p2,p3 data:Selection +└─Selection 2.00 cop[tikv] ge(executor__partition__partition_with_expression.thash.a, 2), le(executor__partition__partition_with_expression.thash.a, 3) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a between 2 and 3; +a b +2 1 +3 2 +SELECT * from t where a < 2; +a b +1 NULL +1 NULL +1 1 +explain format='brief' select * from trange where a < 2; +id estRows task access object operator info +TableReader 3.00 root partition:p0 data:Selection +└─Selection 3.00 cop[tikv] lt(executor__partition__partition_with_expression.trange.a, 2) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a < 2; +a b +1 NULL +1 NULL +1 1 +explain format='brief' select * from thash where a < 2; +id estRows task access object operator info +TableReader 3.00 root partition:all data:Selection +└─Selection 3.00 cop[tikv] lt(executor__partition__partition_with_expression.thash.a, 2) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a < 2; +a b +1 NULL +1 NULL +1 1 +SELECT * from t where a <= 3; +a b +1 NULL +1 NULL +1 1 +2 1 +3 2 +explain format='brief' select * from trange where a <= 3; +id estRows task access object operator info +TableReader 5.00 root partition:p0,p1 data:Selection +└─Selection 5.00 cop[tikv] le(executor__partition__partition_with_expression.trange.a, 3) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a <= 3; +a b +1 NULL +1 NULL +1 1 +2 1 +3 2 +explain format='brief' select * from thash where a <= 3; +id estRows task access object operator info +TableReader 5.00 root partition:all data:Selection +└─Selection 5.00 cop[tikv] le(executor__partition__partition_with_expression.thash.a, 3) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a <= 3; +a b +1 NULL +1 NULL +1 1 +2 1 +3 2 +SELECT * from t where a in (2, 3); +a b +2 1 +3 2 +explain format='brief' select * from trange where a in (2, 3); +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] in(executor__partition__partition_with_expression.trange.a, 2, 3) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a in (2, 3); +a b +2 1 +3 2 +explain format='brief' select * from thash where a in (2, 3); +id estRows task access object operator info +TableReader 2.00 root partition:p2,p3 data:Selection +└─Selection 2.00 cop[tikv] in(executor__partition__partition_with_expression.thash.a, 2, 3) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a in (2, 3); +a b +2 1 +3 2 +SELECT * from t where a in (1, 5); +a b +1 NULL +1 NULL +1 1 +5 5 +explain format='brief' select * from trange where a in (1, 5); +id estRows task access object operator info +TableReader 4.00 root partition:p0,p2 data:Selection +└─Selection 4.00 cop[tikv] in(executor__partition__partition_with_expression.trange.a, 1, 5) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a in (1, 5); +a b +1 NULL +1 NULL +1 1 +5 5 +explain format='brief' select * from thash where a in (1, 5); +id estRows task access object operator info +TableReader 4.00 root partition:p1 data:Selection +└─Selection 4.00 cop[tikv] in(executor__partition__partition_with_expression.thash.a, 1, 5) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a in (1, 5); +a b +1 NULL +1 NULL +1 1 +5 5 +SELECT * from t where a not in (1, 5); +a b +10 NULL +2 1 +3 2 +4 3 +6 7 +7 7 +7 7 +explain format='brief' select * from trange where a not in (1, 5); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] not(in(executor__partition__partition_with_expression.trange.a, 1, 5)) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a not in (1, 5); +a b +10 NULL +2 1 +3 2 +4 3 +6 7 +7 7 +7 7 +explain format='brief' select * from thash where a not in (1, 5); +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] not(in(executor__partition__partition_with_expression.thash.a, 1, 5)) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a not in (1, 5); +a b +10 NULL +2 1 +3 2 +4 3 +6 7 +7 7 +7 7 +SELECT * from t where a = 2 and a = 2; +a b +2 1 +explain format='brief' select * from trange where a = 2 and a = 2; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_with_expression.trange.a, 2) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a = 2 and a = 2; +a b +2 1 +explain format='brief' select * from thash where a = 2 and a = 2; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_with_expression.thash.a, 2) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a = 2 and a = 2; +a b +2 1 +SELECT * from t where a = 2 and a = 3; +a b +explain format='brief' select * from trange where a = 2 and a = 3; +id estRows task access object operator info +TableDual 0.00 root rows:0 +SELECT * from trange where a = 2 and a = 3; +a b +explain format='brief' select * from thash where a = 2 and a = 3; +id estRows task access object operator info +TableDual 0.00 root rows:0 +SELECT * from thash where a = 2 and a = 3; +a b +SELECT * from t where a < 2 and a > 0; +a b +1 NULL +1 NULL +1 1 +explain format='brief' select * from trange where a < 2 and a > 0; +id estRows task access object operator info +TableReader 3.00 root partition:p0 data:Selection +└─Selection 3.00 cop[tikv] gt(executor__partition__partition_with_expression.trange.a, 0), lt(executor__partition__partition_with_expression.trange.a, 2) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a < 2 and a > 0; +a b +1 NULL +1 NULL +1 1 +explain format='brief' select * from thash where a < 2 and a > 0; +id estRows task access object operator info +TableReader 3.00 root partition:p1 data:Selection +└─Selection 3.00 cop[tikv] gt(executor__partition__partition_with_expression.thash.a, 0), lt(executor__partition__partition_with_expression.thash.a, 2) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a < 2 and a > 0; +a b +1 NULL +1 NULL +1 1 +SELECT * from t where a < 2 and a < 3; +a b +1 NULL +1 NULL +1 1 +explain format='brief' select * from trange where a < 2 and a < 3; +id estRows task access object operator info +TableReader 3.00 root partition:p0 data:Selection +└─Selection 3.00 cop[tikv] lt(executor__partition__partition_with_expression.trange.a, 2), lt(executor__partition__partition_with_expression.trange.a, 3) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a < 2 and a < 3; +a b +1 NULL +1 NULL +1 1 +explain format='brief' select * from thash where a < 2 and a < 3; +id estRows task access object operator info +TableReader 3.00 root partition:all data:Selection +└─Selection 3.00 cop[tikv] lt(executor__partition__partition_with_expression.thash.a, 2), lt(executor__partition__partition_with_expression.thash.a, 3) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a < 2 and a < 3; +a b +1 NULL +1 NULL +1 1 +SELECT * from t where a > 1 and a > 2; +a b +10 NULL +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from trange where a > 1 and a > 2; +id estRows task access object operator info +TableReader 7.00 root partition:p1,p2 data:Selection +└─Selection 7.00 cop[tikv] gt(executor__partition__partition_with_expression.trange.a, 1), gt(executor__partition__partition_with_expression.trange.a, 2) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a > 1 and a > 2; +a b +10 NULL +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from thash where a > 1 and a > 2; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] gt(executor__partition__partition_with_expression.thash.a, 1), gt(executor__partition__partition_with_expression.thash.a, 2) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a > 1 and a > 2; +a b +10 NULL +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +SELECT * from t where a = 2 or a = 3; +a b +2 1 +3 2 +explain format='brief' select * from trange where a = 2 or a = 3; +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] or(eq(executor__partition__partition_with_expression.trange.a, 2), eq(executor__partition__partition_with_expression.trange.a, 3)) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a = 2 or a = 3; +a b +2 1 +3 2 +explain format='brief' select * from thash where a = 2 or a = 3; +id estRows task access object operator info +TableReader 2.00 root partition:p2,p3 data:Selection +└─Selection 2.00 cop[tikv] or(eq(executor__partition__partition_with_expression.thash.a, 2), eq(executor__partition__partition_with_expression.thash.a, 3)) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a = 2 or a = 3; +a b +2 1 +3 2 +SELECT * from t where a = 2 or a in (3); +a b +2 1 +3 2 +explain format='brief' select * from trange where a = 2 or a in (3); +id estRows task access object operator info +TableReader 2.00 root partition:p0,p1 data:Selection +└─Selection 2.00 cop[tikv] or(eq(executor__partition__partition_with_expression.trange.a, 2), eq(executor__partition__partition_with_expression.trange.a, 3)) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a = 2 or a in (3); +a b +2 1 +3 2 +explain format='brief' select * from thash where a = 2 or a in (3); +id estRows task access object operator info +TableReader 2.00 root partition:p2,p3 data:Selection +└─Selection 2.00 cop[tikv] or(eq(executor__partition__partition_with_expression.thash.a, 2), eq(executor__partition__partition_with_expression.thash.a, 3)) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a = 2 or a in (3); +a b +2 1 +3 2 +SELECT * from t where a = 2 or a > 3; +a b +10 NULL +2 1 +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from trange where a = 2 or a > 3; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(eq(executor__partition__partition_with_expression.trange.a, 2), gt(executor__partition__partition_with_expression.trange.a, 3)) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a = 2 or a > 3; +a b +10 NULL +2 1 +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from thash where a = 2 or a > 3; +id estRows task access object operator info +TableReader 7.00 root partition:all data:Selection +└─Selection 7.00 cop[tikv] or(eq(executor__partition__partition_with_expression.thash.a, 2), gt(executor__partition__partition_with_expression.thash.a, 3)) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a = 2 or a > 3; +a b +10 NULL +2 1 +4 3 +5 5 +6 7 +7 7 +7 7 +SELECT * from t where a = 2 or a <= 1; +a b +1 NULL +1 NULL +1 1 +2 1 +explain format='brief' select * from trange where a = 2 or a <= 1; +id estRows task access object operator info +TableReader 4.00 root partition:p0 data:Selection +└─Selection 4.00 cop[tikv] or(eq(executor__partition__partition_with_expression.trange.a, 2), le(executor__partition__partition_with_expression.trange.a, 1)) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a = 2 or a <= 1; +a b +1 NULL +1 NULL +1 1 +2 1 +explain format='brief' select * from thash where a = 2 or a <= 1; +id estRows task access object operator info +TableReader 4.00 root partition:all data:Selection +└─Selection 4.00 cop[tikv] or(eq(executor__partition__partition_with_expression.thash.a, 2), le(executor__partition__partition_with_expression.thash.a, 1)) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a = 2 or a <= 1; +a b +1 NULL +1 NULL +1 1 +2 1 +SELECT * from t where a = 2 or a between 2 and 2; +a b +2 1 +explain format='brief' select * from trange where a = 2 or a between 2 and 2; +id estRows task access object operator info +TableReader 1.00 root partition:p0 data:Selection +└─Selection 1.00 cop[tikv] or(eq(executor__partition__partition_with_expression.trange.a, 2), and(ge(executor__partition__partition_with_expression.trange.a, 2), le(executor__partition__partition_with_expression.trange.a, 2))) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a = 2 or a between 2 and 2; +a b +2 1 +explain format='brief' select * from thash where a = 2 or a between 2 and 2; +id estRows task access object operator info +TableReader 1.00 root partition:p2 data:Selection +└─Selection 1.00 cop[tikv] or(eq(executor__partition__partition_with_expression.thash.a, 2), and(ge(executor__partition__partition_with_expression.thash.a, 2), le(executor__partition__partition_with_expression.thash.a, 2))) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a = 2 or a between 2 and 2; +a b +2 1 +SELECT * from t where a != 2; +a b +1 NULL +1 NULL +1 1 +10 NULL +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from trange where a != 2; +id estRows task access object operator info +TableReader 10.00 root partition:all data:Selection +└─Selection 10.00 cop[tikv] ne(executor__partition__partition_with_expression.trange.a, 2) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a != 2; +a b +1 NULL +1 NULL +1 1 +10 NULL +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from thash where a != 2; +id estRows task access object operator info +TableReader 10.00 root partition:all data:Selection +└─Selection 10.00 cop[tikv] ne(executor__partition__partition_with_expression.thash.a, 2) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a != 2; +a b +1 NULL +1 NULL +1 1 +10 NULL +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +SELECT * from t where a != 2 and a > 4; +a b +10 NULL +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from trange where a != 2 and a > 4; +id estRows task access object operator info +TableReader 5.00 root partition:p2 data:Selection +└─Selection 5.00 cop[tikv] gt(executor__partition__partition_with_expression.trange.a, 4), ne(executor__partition__partition_with_expression.trange.a, 2) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a != 2 and a > 4; +a b +10 NULL +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from thash where a != 2 and a > 4; +id estRows task access object operator info +TableReader 5.00 root partition:all data:Selection +└─Selection 5.00 cop[tikv] gt(executor__partition__partition_with_expression.thash.a, 4), ne(executor__partition__partition_with_expression.thash.a, 2) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a != 2 and a > 4; +a b +10 NULL +5 5 +6 7 +7 7 +7 7 +SELECT * from t where a != 2 and a != 3; +a b +1 NULL +1 NULL +1 1 +10 NULL +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from trange where a != 2 and a != 3; +id estRows task access object operator info +TableReader 9.00 root partition:all data:Selection +└─Selection 9.00 cop[tikv] ne(executor__partition__partition_with_expression.trange.a, 2), ne(executor__partition__partition_with_expression.trange.a, 3) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a != 2 and a != 3; +a b +1 NULL +1 NULL +1 1 +10 NULL +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from thash where a != 2 and a != 3; +id estRows task access object operator info +TableReader 9.00 root partition:all data:Selection +└─Selection 9.00 cop[tikv] ne(executor__partition__partition_with_expression.thash.a, 2), ne(executor__partition__partition_with_expression.thash.a, 3) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a != 2 and a != 3; +a b +1 NULL +1 NULL +1 1 +10 NULL +4 3 +5 5 +6 7 +7 7 +7 7 +SELECT * from t where a != 2 and a = 3; +a b +3 2 +explain format='brief' select * from trange where a != 2 and a = 3; +id estRows task access object operator info +TableReader 1.00 root partition:p1 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_with_expression.trange.a, 3) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a != 2 and a = 3; +a b +3 2 +explain format='brief' select * from thash where a != 2 and a = 3; +id estRows task access object operator info +TableReader 1.00 root partition:p3 data:Selection +└─Selection 1.00 cop[tikv] eq(executor__partition__partition_with_expression.thash.a, 3) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a != 2 and a = 3; +a b +3 2 +SELECT * from t where not (a = 2); +a b +1 NULL +1 NULL +1 1 +10 NULL +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from trange where not (a = 2); +id estRows task access object operator info +TableReader 10.00 root partition:all data:Selection +└─Selection 10.00 cop[tikv] ne(executor__partition__partition_with_expression.trange.a, 2) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where not (a = 2); +a b +1 NULL +1 NULL +1 1 +10 NULL +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from thash where not (a = 2); +id estRows task access object operator info +TableReader 10.00 root partition:all data:Selection +└─Selection 10.00 cop[tikv] ne(executor__partition__partition_with_expression.thash.a, 2) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where not (a = 2); +a b +1 NULL +1 NULL +1 1 +10 NULL +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +SELECT * from t where not (a > 2); +a b +1 NULL +1 NULL +1 1 +2 1 +explain format='brief' select * from trange where not (a > 2); +id estRows task access object operator info +TableReader 4.00 root partition:p0 data:Selection +└─Selection 4.00 cop[tikv] le(executor__partition__partition_with_expression.trange.a, 2) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where not (a > 2); +a b +1 NULL +1 NULL +1 1 +2 1 +explain format='brief' select * from thash where not (a > 2); +id estRows task access object operator info +TableReader 4.00 root partition:all data:Selection +└─Selection 4.00 cop[tikv] le(executor__partition__partition_with_expression.thash.a, 2) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where not (a > 2); +a b +1 NULL +1 NULL +1 1 +2 1 +SELECT * from t where not (a < 2); +a b +10 NULL +2 1 +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from trange where not (a < 2); +id estRows task access object operator info +TableReader 8.00 root partition:all data:Selection +└─Selection 8.00 cop[tikv] ge(executor__partition__partition_with_expression.trange.a, 2) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where not (a < 2); +a b +10 NULL +2 1 +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from thash where not (a < 2); +id estRows task access object operator info +TableReader 8.00 root partition:all data:Selection +└─Selection 8.00 cop[tikv] ge(executor__partition__partition_with_expression.thash.a, 2) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where not (a < 2); +a b +10 NULL +2 1 +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +SELECT * from t where a + 1 > 4; +a b +10 NULL +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from trange where a + 1 > 4; +id estRows task access object operator info +TableReader 10.40 root partition:all data:Selection +└─Selection 10.40 cop[tikv] gt(plus(executor__partition__partition_with_expression.trange.a, 1), 4) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a + 1 > 4; +a b +10 NULL +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from thash where a + 1 > 4; +id estRows task access object operator info +TableReader 10.40 root partition:all data:Selection +└─Selection 10.40 cop[tikv] gt(plus(executor__partition__partition_with_expression.thash.a, 1), 4) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a + 1 > 4; +a b +10 NULL +4 3 +5 5 +6 7 +7 7 +7 7 +SELECT * from t where a - 1 > 0; +a b +10 NULL +2 1 +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from trange where a - 1 > 0; +id estRows task access object operator info +TableReader 10.40 root partition:all data:Selection +└─Selection 10.40 cop[tikv] gt(minus(executor__partition__partition_with_expression.trange.a, 1), 0) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a - 1 > 0; +a b +10 NULL +2 1 +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +explain format='brief' select * from thash where a - 1 > 0; +id estRows task access object operator info +TableReader 10.40 root partition:all data:Selection +└─Selection 10.40 cop[tikv] gt(minus(executor__partition__partition_with_expression.thash.a, 1), 0) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a - 1 > 0; +a b +10 NULL +2 1 +3 2 +4 3 +5 5 +6 7 +7 7 +7 7 +SELECT * from t where a * 2 < 0; +a b +explain format='brief' select * from trange where a * 2 < 0; +id estRows task access object operator info +TableReader 10.40 root partition:all data:Selection +└─Selection 10.40 cop[tikv] lt(mul(executor__partition__partition_with_expression.trange.a, 2), 0) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a * 2 < 0; +a b +explain format='brief' select * from thash where a * 2 < 0; +id estRows task access object operator info +TableReader 10.40 root partition:all data:Selection +└─Selection 10.40 cop[tikv] lt(mul(executor__partition__partition_with_expression.thash.a, 2), 0) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a * 2 < 0; +a b +SELECT * from t where a << 1 < 0; +a b +explain format='brief' select * from trange where a << 1 < 0; +id estRows task access object operator info +TableReader 10.40 root partition:all data:Selection +└─Selection 10.40 cop[tikv] lt(leftshift(executor__partition__partition_with_expression.trange.a, 1), 0) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a << 1 < 0; +a b +explain format='brief' select * from thash where a << 1 < 0; +id estRows task access object operator info +TableReader 10.40 root partition:all data:Selection +└─Selection 10.40 cop[tikv] lt(leftshift(executor__partition__partition_with_expression.thash.a, 1), 0) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a << 1 < 0; +a b +SELECT * from t where a > '10'; +a b +explain format='brief' select * from trange where a > '10'; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_with_expression.trange.a, 10) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a > '10'; +a b +explain format='brief' select * from thash where a > '10'; +id estRows task access object operator info +TableReader 1.00 root partition:all data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_with_expression.thash.a, 10) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a > '10'; +a b +SELECT * from t where a > '10ab'; +a b +explain format='brief' select * from trange where a > '10ab'; +id estRows task access object operator info +TableReader 1.00 root partition:dual data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_with_expression.trange.a, 10) + └─TableFullScan 13.00 cop[tikv] table:trange keep order:false +SELECT * from trange where a > '10ab'; +a b +explain format='brief' select * from thash where a > '10ab'; +id estRows task access object operator info +TableReader 1.00 root partition:all data:Selection +└─Selection 1.00 cop[tikv] gt(executor__partition__partition_with_expression.thash.a, 10) + └─TableFullScan 13.00 cop[tikv] table:thash keep order:false +SELECT * from thash where a > '10ab'; +a b +set tidb_partition_prune_mode=default; diff --git a/tests/integrationtest/r/planner/cardinality/selectivity.result b/tests/integrationtest/r/planner/cardinality/selectivity.result new file mode 100644 index 0000000000000..89bcbb5ce9da3 --- /dev/null +++ b/tests/integrationtest/r/planner/cardinality/selectivity.result @@ -0,0 +1,1424 @@ +drop table if exists t; +set tidb_enable_clustered_index = 'INT_ONLY'; +create table t(a char(10) primary key, b int); +explain select * from t where a > "t"; +id estRows task access object operator info +TableReader_7 3333.33 root data:Selection_6 +└─Selection_6 3333.33 cop[tikv] gt(planner__cardinality__selectivity.t.a, "t") + └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +drop table t; +create table t(a int primary key, b int); +explain select * from t where a > 1; +id estRows task access object operator info +TableReader_6 3333.33 root data:TableRangeScan_5 +└─TableRangeScan_5 3333.33 cop[tikv] table:t range:(1,+inf], keep order:false, stats:pseudo +set tidb_enable_clustered_index = DEFAULT; +drop table t; +set tidb_cost_model_version=2; +set tidb_analyze_version=2; +drop table if exists tint; +create table tint(a int, b int, c int, index singular(a), index multi(b, c)); +insert into tint values (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5), (6, 6, 6), (7, 7, 7), (8, 8, 8); +analyze table tint all columns with 2 topn, 3 buckets; +drop table if exists tdouble; +create table tdouble(a double, b double, c double, index singular(a), index multi(b, c)); +insert into tdouble values (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5), (6, 6, 6), (7, 7, 7), (8, 8, 8); +analyze table tdouble all columns with 2 topn, 3 buckets; +drop table if exists tdecimal; +create table tdecimal(a decimal(40, 20), b decimal(40, 20), c decimal(40, 20), index singular(a), index multi(b, c)); +insert into tdecimal values (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5), (6, 6, 6), (7, 7, 7), (8, 8, 8); +analyze table tdecimal all columns with 2 topn, 3 buckets; +drop table if exists tstring; +create table tstring(a varchar(64), b varchar(64), c varchar(64), index singular(a), index multi(b, c)); +insert into tstring values ('1', '1', '1'), ('2', '2', '2'), ('3', '3', '3'), ('4', '4', '4'), ('5', '5', '5'), ('6', '6', '6'), ('7', '7', '7'), ('8', '8', '8'); +analyze table tstring all columns with 2 topn, 3 buckets; +drop table if exists tdatetime; +create table tdatetime(a datetime, b datetime, c datetime, index singular(a), index multi(b, c)); +insert into tdatetime values ('2001-01-01', '2001-01-01', '2001-01-01'), ('2001-01-02', '2001-01-02', '2001-01-02'), ('2001-01-03', '2001-01-03', '2001-01-03'), ('2001-01-04', '2001-01-04', '2001-01-04'); +analyze table tdatetime all columns with 2 topn, 3 buckets; +drop table if exists tprefix; +create table tprefix(a varchar(64), b varchar(64), index prefixa(a(2))); +insert into tprefix values ('111', '111'), ('222', '222'), ('333', '333'), ('444', '444'), ('555', '555'), ('666', '666'); +analyze table tprefix all columns with 2 topn, 3 buckets; +drop table if exists ct1; +create table ct1 (a int, pk varchar(10), primary key(pk) clustered); +insert into ct1 values (1, '1'), (2, '2'), (3, '3'), (4, '4'), (5, '5'), (6, '6'), (7, '7'), (8, '8'); +analyze table ct1 all columns with 2 topn, 3 buckets; +drop table if exists ct2; +create table ct2 (a int, b int, c int, primary key(a, b) clustered); +insert into ct2 values (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5), (6, 6, 6), (7, 7, 7), (8, 8, 8); +analyze table ct2 all columns with 2 topn, 3 buckets; +select stats_ver from mysql.stats_histograms where table_id in (select TIDB_TABLE_ID from information_schema.tables where TABLE_SCHEMA = 'planner__cardinality__selectivity'); +stats_ver +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +show stats_topn where db_name = 'planner__cardinality__selectivity'; +Db_name Table_name Partition_name Column_name Is_index Value Count +planner__cardinality__selectivity ct1 PRIMARY 1 1 1 +planner__cardinality__selectivity ct1 PRIMARY 1 2 1 +planner__cardinality__selectivity ct1 a 0 1 1 +planner__cardinality__selectivity ct1 a 0 2 1 +planner__cardinality__selectivity ct1 pk 0 1 1 +planner__cardinality__selectivity ct1 pk 0 2 1 +planner__cardinality__selectivity ct2 PRIMARY 1 (1, 1) 1 +planner__cardinality__selectivity ct2 PRIMARY 1 (2, 2) 1 +planner__cardinality__selectivity ct2 a 0 1 1 +planner__cardinality__selectivity ct2 a 0 2 1 +planner__cardinality__selectivity ct2 b 0 1 1 +planner__cardinality__selectivity ct2 b 0 2 1 +planner__cardinality__selectivity ct2 c 0 1 1 +planner__cardinality__selectivity ct2 c 0 2 1 +planner__cardinality__selectivity tdatetime a 0 2001-01-01 00:00:00 1 +planner__cardinality__selectivity tdatetime a 0 2001-01-02 00:00:00 1 +planner__cardinality__selectivity tdatetime b 0 2001-01-01 00:00:00 1 +planner__cardinality__selectivity tdatetime b 0 2001-01-02 00:00:00 1 +planner__cardinality__selectivity tdatetime c 0 2001-01-01 00:00:00 1 +planner__cardinality__selectivity tdatetime c 0 2001-01-02 00:00:00 1 +planner__cardinality__selectivity tdatetime multi 1 (2001-01-01 00:00:00, 2001-01-01 00:00:00) 1 +planner__cardinality__selectivity tdatetime multi 1 (2001-01-02 00:00:00, 2001-01-02 00:00:00) 1 +planner__cardinality__selectivity tdatetime singular 1 2001-01-01 00:00:00 1 +planner__cardinality__selectivity tdatetime singular 1 2001-01-02 00:00:00 1 +planner__cardinality__selectivity tdecimal a 0 1.00000000000000000000 1 +planner__cardinality__selectivity tdecimal a 0 2.00000000000000000000 1 +planner__cardinality__selectivity tdecimal b 0 1.00000000000000000000 1 +planner__cardinality__selectivity tdecimal b 0 2.00000000000000000000 1 +planner__cardinality__selectivity tdecimal c 0 1.00000000000000000000 1 +planner__cardinality__selectivity tdecimal c 0 2.00000000000000000000 1 +planner__cardinality__selectivity tdecimal multi 1 (1.00000000000000000000, 1.00000000000000000000) 1 +planner__cardinality__selectivity tdecimal multi 1 (2.00000000000000000000, 2.00000000000000000000) 1 +planner__cardinality__selectivity tdecimal singular 1 1.00000000000000000000 1 +planner__cardinality__selectivity tdecimal singular 1 2.00000000000000000000 1 +planner__cardinality__selectivity tdouble a 0 1 1 +planner__cardinality__selectivity tdouble a 0 2 1 +planner__cardinality__selectivity tdouble b 0 1 1 +planner__cardinality__selectivity tdouble b 0 2 1 +planner__cardinality__selectivity tdouble c 0 1 1 +planner__cardinality__selectivity tdouble c 0 2 1 +planner__cardinality__selectivity tdouble multi 1 (1, 1) 1 +planner__cardinality__selectivity tdouble multi 1 (2, 2) 1 +planner__cardinality__selectivity tdouble singular 1 1 1 +planner__cardinality__selectivity tdouble singular 1 2 1 +planner__cardinality__selectivity tint a 0 1 1 +planner__cardinality__selectivity tint a 0 2 1 +planner__cardinality__selectivity tint b 0 1 1 +planner__cardinality__selectivity tint b 0 2 1 +planner__cardinality__selectivity tint c 0 1 1 +planner__cardinality__selectivity tint c 0 2 1 +planner__cardinality__selectivity tint multi 1 (1, 1) 1 +planner__cardinality__selectivity tint multi 1 (2, 2) 1 +planner__cardinality__selectivity tint singular 1 1 1 +planner__cardinality__selectivity tint singular 1 2 1 +planner__cardinality__selectivity tprefix a 0 111 1 +planner__cardinality__selectivity tprefix a 0 222 1 +planner__cardinality__selectivity tprefix b 0 111 1 +planner__cardinality__selectivity tprefix b 0 222 1 +planner__cardinality__selectivity tprefix prefixa 1 11 1 +planner__cardinality__selectivity tprefix prefixa 1 22 1 +planner__cardinality__selectivity tstring a 0 1 1 +planner__cardinality__selectivity tstring a 0 2 1 +planner__cardinality__selectivity tstring b 0 1 1 +planner__cardinality__selectivity tstring b 0 2 1 +planner__cardinality__selectivity tstring c 0 1 1 +planner__cardinality__selectivity tstring c 0 2 1 +planner__cardinality__selectivity tstring multi 1 (1, 1) 1 +planner__cardinality__selectivity tstring multi 1 (2, 2) 1 +planner__cardinality__selectivity tstring singular 1 1 1 +planner__cardinality__selectivity tstring singular 1 2 1 +show stats_buckets where db_name = 'planner__cardinality__selectivity'; +Db_name Table_name Partition_name Column_name Is_index Bucket_id Count Repeats Lower_Bound Upper_Bound Ndv +planner__cardinality__selectivity ct1 PRIMARY 1 0 3 1 3 5 0 +planner__cardinality__selectivity ct1 PRIMARY 1 1 6 1 6 8 0 +planner__cardinality__selectivity ct1 a 0 0 3 1 3 5 0 +planner__cardinality__selectivity ct1 a 0 1 6 1 6 8 0 +planner__cardinality__selectivity ct1 pk 0 0 3 1 3 5 0 +planner__cardinality__selectivity ct1 pk 0 1 6 1 6 8 0 +planner__cardinality__selectivity ct2 PRIMARY 1 0 3 1 (3, 3) (5, 5) 0 +planner__cardinality__selectivity ct2 PRIMARY 1 1 6 1 (6, 6) (8, 8) 0 +planner__cardinality__selectivity ct2 a 0 0 3 1 3 5 0 +planner__cardinality__selectivity ct2 a 0 1 6 1 6 8 0 +planner__cardinality__selectivity ct2 b 0 0 3 1 3 5 0 +planner__cardinality__selectivity ct2 b 0 1 6 1 6 8 0 +planner__cardinality__selectivity ct2 c 0 0 3 1 3 5 0 +planner__cardinality__selectivity ct2 c 0 1 6 1 6 8 0 +planner__cardinality__selectivity tdatetime a 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0 +planner__cardinality__selectivity tdatetime a 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0 +planner__cardinality__selectivity tdatetime b 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0 +planner__cardinality__selectivity tdatetime b 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0 +planner__cardinality__selectivity tdatetime c 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0 +planner__cardinality__selectivity tdatetime c 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0 +planner__cardinality__selectivity tdatetime multi 1 0 1 1 (2001-01-03 00:00:00, 2001-01-03 00:00:00) (2001-01-03 00:00:00, 2001-01-03 00:00:00) 0 +planner__cardinality__selectivity tdatetime multi 1 1 2 1 (2001-01-04 00:00:00, 2001-01-04 00:00:00) (2001-01-04 00:00:00, 2001-01-04 00:00:00) 0 +planner__cardinality__selectivity tdatetime singular 1 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0 +planner__cardinality__selectivity tdatetime singular 1 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0 +planner__cardinality__selectivity tdecimal a 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0 +planner__cardinality__selectivity tdecimal a 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0 +planner__cardinality__selectivity tdecimal b 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0 +planner__cardinality__selectivity tdecimal b 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0 +planner__cardinality__selectivity tdecimal c 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0 +planner__cardinality__selectivity tdecimal c 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0 +planner__cardinality__selectivity tdecimal multi 1 0 3 1 (3.00000000000000000000, 3.00000000000000000000) (5.00000000000000000000, 5.00000000000000000000) 0 +planner__cardinality__selectivity tdecimal multi 1 1 6 1 (6.00000000000000000000, 6.00000000000000000000) (8.00000000000000000000, 8.00000000000000000000) 0 +planner__cardinality__selectivity tdecimal singular 1 0 3 1 3.00000000000000000000 5.00000000000000000000 0 +planner__cardinality__selectivity tdecimal singular 1 1 6 1 6.00000000000000000000 8.00000000000000000000 0 +planner__cardinality__selectivity tdouble a 0 0 3 1 3 5 0 +planner__cardinality__selectivity tdouble a 0 1 6 1 6 8 0 +planner__cardinality__selectivity tdouble b 0 0 3 1 3 5 0 +planner__cardinality__selectivity tdouble b 0 1 6 1 6 8 0 +planner__cardinality__selectivity tdouble c 0 0 3 1 3 5 0 +planner__cardinality__selectivity tdouble c 0 1 6 1 6 8 0 +planner__cardinality__selectivity tdouble multi 1 0 3 1 (3, 3) (5, 5) 0 +planner__cardinality__selectivity tdouble multi 1 1 6 1 (6, 6) (8, 8) 0 +planner__cardinality__selectivity tdouble singular 1 0 3 1 3 5 0 +planner__cardinality__selectivity tdouble singular 1 1 6 1 6 8 0 +planner__cardinality__selectivity tint a 0 0 3 1 3 5 0 +planner__cardinality__selectivity tint a 0 1 6 1 6 8 0 +planner__cardinality__selectivity tint b 0 0 3 1 3 5 0 +planner__cardinality__selectivity tint b 0 1 6 1 6 8 0 +planner__cardinality__selectivity tint c 0 0 3 1 3 5 0 +planner__cardinality__selectivity tint c 0 1 6 1 6 8 0 +planner__cardinality__selectivity tint multi 1 0 3 1 (3, 3) (5, 5) 0 +planner__cardinality__selectivity tint multi 1 1 6 1 (6, 6) (8, 8) 0 +planner__cardinality__selectivity tint singular 1 0 3 1 3 5 0 +planner__cardinality__selectivity tint singular 1 1 6 1 6 8 0 +planner__cardinality__selectivity tprefix a 0 0 2 1 333 444 0 +planner__cardinality__selectivity tprefix a 0 1 4 1 555 666 0 +planner__cardinality__selectivity tprefix b 0 0 2 1 333 444 0 +planner__cardinality__selectivity tprefix b 0 1 4 1 555 666 0 +planner__cardinality__selectivity tprefix prefixa 1 0 2 1 33 44 0 +planner__cardinality__selectivity tprefix prefixa 1 1 4 1 55 66 0 +planner__cardinality__selectivity tstring a 0 0 3 1 3 5 0 +planner__cardinality__selectivity tstring a 0 1 6 1 6 8 0 +planner__cardinality__selectivity tstring b 0 0 3 1 3 5 0 +planner__cardinality__selectivity tstring b 0 1 6 1 6 8 0 +planner__cardinality__selectivity tstring c 0 0 3 1 3 5 0 +planner__cardinality__selectivity tstring c 0 1 6 1 6 8 0 +planner__cardinality__selectivity tstring multi 1 0 3 1 (3, 3) (5, 5) 0 +planner__cardinality__selectivity tstring multi 1 1 6 1 (6, 6) (8, 8) 0 +planner__cardinality__selectivity tstring singular 1 0 3 1 3 5 0 +planner__cardinality__selectivity tstring singular 1 1 6 1 6 8 0 +explain select * from tint where a=1; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tint.a, 1) + └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false +explain select * from tint where a=4; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tint.a, 4) + └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false +explain select * from tint where a=8; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tint.a, 8) + └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false +explain select * from tdouble where a=1; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdouble.a, 1) + └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false +explain select * from tdouble where a=4; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdouble.a, 4) + └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false +explain select * from tdouble where a=8; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdouble.a, 8) + └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false +explain select * from tdecimal where a=1; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdecimal.a, 1) + └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false +explain select * from tdecimal where a=4; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdecimal.a, 4) + └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false +explain select * from tdecimal where a=8; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdecimal.a, 8) + └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false +explain select * from tstring where a='1'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tstring.a, "1") + └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false +explain select * from tstring where a='4'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tstring.a, "4") + └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false +explain select * from tstring where a='8'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tstring.a, "8") + └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false +explain select * from tdatetime where a='2001-01-01'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdatetime.a, 2001-01-01 00:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false +explain select * from tdatetime where a='2001-01-02'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdatetime.a, 2001-01-02 00:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false +explain select * from tdatetime where a='2001-01-04'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdatetime.a, 2001-01-04 00:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false +explain select * from tprefix where a='111'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tprefix.a, "111") + └─TableFullScan_5 6.00 cop[tikv] table:tprefix keep order:false +explain select * from tprefix where a='444'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tprefix.a, "444") + └─TableFullScan_5 6.00 cop[tikv] table:tprefix keep order:false +explain select * from tprefix where a='888'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tprefix.a, "888") + └─TableFullScan_5 6.00 cop[tikv] table:tprefix keep order:false +explain select * from tint where b=1 and c=1; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tint.b, 1), eq(planner__cardinality__selectivity.tint.c, 1) + └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false +explain select * from tint where b=4 and c=4; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tint.b, 4), eq(planner__cardinality__selectivity.tint.c, 4) + └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false +explain select * from tint where b=8 and c=8; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tint.b, 8), eq(planner__cardinality__selectivity.tint.c, 8) + └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false +explain select * from tdouble where b=1 and c=1; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdouble.b, 1), eq(planner__cardinality__selectivity.tdouble.c, 1) + └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false +explain select * from tdouble where b=4 and c=4; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdouble.b, 4), eq(planner__cardinality__selectivity.tdouble.c, 4) + └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false +explain select * from tdouble where b=8 and c=8; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdouble.b, 8), eq(planner__cardinality__selectivity.tdouble.c, 8) + └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false +explain select * from tdecimal where b=1 and c=1; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdecimal.b, 1), eq(planner__cardinality__selectivity.tdecimal.c, 1) + └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false +explain select * from tdecimal where b=4 and c=4; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdecimal.b, 4), eq(planner__cardinality__selectivity.tdecimal.c, 4) + └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false +explain select * from tdecimal where b=8 and c=8; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdecimal.b, 8), eq(planner__cardinality__selectivity.tdecimal.c, 8) + └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false +explain select * from tstring where b='1' and c='1'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tstring.b, "1"), eq(planner__cardinality__selectivity.tstring.c, "1") + └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false +explain select * from tstring where b='4' and c='4'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tstring.b, "4"), eq(planner__cardinality__selectivity.tstring.c, "4") + └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false +explain select * from tstring where b='8' and c='8'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tstring.b, "8"), eq(planner__cardinality__selectivity.tstring.c, "8") + └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false +explain select * from tdatetime where b='2001-01-01' and c='2001-01-01'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdatetime.b, 2001-01-01 00:00:00.000000), eq(planner__cardinality__selectivity.tdatetime.c, 2001-01-01 00:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false +explain select * from tdatetime where b='2001-01-02' and c='2001-01-02'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdatetime.b, 2001-01-02 00:00:00.000000), eq(planner__cardinality__selectivity.tdatetime.c, 2001-01-02 00:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false +explain select * from tdatetime where b='2001-01-04' and c='2001-01-04'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdatetime.b, 2001-01-04 00:00:00.000000), eq(planner__cardinality__selectivity.tdatetime.c, 2001-01-04 00:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false +explain select * from tint where b=1; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tint.b, 1) + └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false +explain select * from tint where b=4; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tint.b, 4) + └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false +explain select * from tint where b=8; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tint.b, 8) + └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false +explain select * from tdouble where b=1; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdouble.b, 1) + └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false +explain select * from tdouble where b=4; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdouble.b, 4) + └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false +explain select * from tdouble where b=8; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdouble.b, 8) + └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false +explain select * from tdecimal where b=1; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdecimal.b, 1) + └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false +explain select * from tdecimal where b=4; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdecimal.b, 4) + └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false +explain select * from tdecimal where b=8; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdecimal.b, 8) + └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false +explain select * from tstring where b='1'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tstring.b, "1") + └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false +explain select * from tstring where b='4'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tstring.b, "4") + └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false +explain select * from tstring where b='8'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tstring.b, "8") + └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false +explain select * from tdatetime where b='2001-01-01'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdatetime.b, 2001-01-01 00:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false +explain select * from tdatetime where b='2001-01-02'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdatetime.b, 2001-01-02 00:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false +explain select * from tdatetime where b='2001-01-04'; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.tdatetime.b, 2001-01-04 00:00:00.000000) + └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false +explain select * from ct1 where pk>='1' and pk <='4'; +id estRows task access object operator info +TableReader_6 5.00 root data:TableRangeScan_5 +└─TableRangeScan_5 5.00 cop[tikv] table:ct1 range:["1","4"], keep order:false +explain select * from ct1 where pk>='4' and pk <='6'; +id estRows task access object operator info +TableReader_6 3.75 root data:TableRangeScan_5 +└─TableRangeScan_5 3.75 cop[tikv] table:ct1 range:["4","6"], keep order:false +explain select * from ct1 where pk>='6' and pk <='8'; +id estRows task access object operator info +TableReader_6 3.00 root data:TableRangeScan_5 +└─TableRangeScan_5 3.00 cop[tikv] table:ct1 range:["6","8"], keep order:false +explain select * from ct2 where a=1 and b>=1 and b<=8; +id estRows task access object operator info +TableReader_6 1.00 root data:TableRangeScan_5 +└─TableRangeScan_5 1.00 cop[tikv] table:ct2 range:[1 1,1 8], keep order:false +explain select * from ct2 where a=4 and b>=1 and b<=8; +id estRows task access object operator info +TableReader_6 1.00 root data:TableRangeScan_5 +└─TableRangeScan_5 1.00 cop[tikv] table:ct2 range:[4 1,4 8], keep order:false +explain select * from ct2 where a=8 and b>=1 and b<=8; +id estRows task access object operator info +TableReader_6 1.00 root data:TableRangeScan_5 +└─TableRangeScan_5 1.00 cop[tikv] table:ct2 range:[8 1,8 8], keep order:false +set tidb_analyze_version=2; +drop table if exists topn_before_hist; +create table topn_before_hist(a int, index idx(a)); +insert into topn_before_hist values(1), (1), (1), (1), (3), (3), (4), (5), (6); +analyze table topn_before_hist all columns with 2 topn, 3 buckets; +create table topn_after_hist(a int, index idx(a)); +insert into topn_after_hist values(2), (2), (3), (4), (5), (7), (7), (7), (7); +analyze table topn_after_hist all columns with 2 topn, 3 buckets; +create table topn_before_hist_no_index(a int); +insert into topn_before_hist_no_index values(1), (1), (1), (1), (3), (3), (4), (5), (6); +analyze table topn_before_hist_no_index all columns with 2 topn, 3 buckets; +create table topn_after_hist_no_index(a int); +insert into topn_after_hist_no_index values(2), (2), (3), (4), (5), (7), (7), (7), (7); +analyze table topn_after_hist_no_index all columns with 2 topn, 3 buckets; +show stats_topn where db_name = 'planner__cardinality__selectivity'; +Db_name Table_name Partition_name Column_name Is_index Value Count +planner__cardinality__selectivity ct1 PRIMARY 1 1 1 +planner__cardinality__selectivity ct1 PRIMARY 1 2 1 +planner__cardinality__selectivity ct1 a 0 1 1 +planner__cardinality__selectivity ct1 a 0 2 1 +planner__cardinality__selectivity ct1 pk 0 1 1 +planner__cardinality__selectivity ct1 pk 0 2 1 +planner__cardinality__selectivity ct2 PRIMARY 1 (1, 1) 1 +planner__cardinality__selectivity ct2 PRIMARY 1 (2, 2) 1 +planner__cardinality__selectivity ct2 a 0 1 1 +planner__cardinality__selectivity ct2 a 0 2 1 +planner__cardinality__selectivity ct2 b 0 1 1 +planner__cardinality__selectivity ct2 b 0 2 1 +planner__cardinality__selectivity ct2 c 0 1 1 +planner__cardinality__selectivity ct2 c 0 2 1 +planner__cardinality__selectivity tdatetime a 0 2001-01-01 00:00:00 1 +planner__cardinality__selectivity tdatetime a 0 2001-01-02 00:00:00 1 +planner__cardinality__selectivity tdatetime b 0 2001-01-01 00:00:00 1 +planner__cardinality__selectivity tdatetime b 0 2001-01-02 00:00:00 1 +planner__cardinality__selectivity tdatetime c 0 2001-01-01 00:00:00 1 +planner__cardinality__selectivity tdatetime c 0 2001-01-02 00:00:00 1 +planner__cardinality__selectivity tdatetime multi 1 (2001-01-01 00:00:00, 2001-01-01 00:00:00) 1 +planner__cardinality__selectivity tdatetime multi 1 (2001-01-02 00:00:00, 2001-01-02 00:00:00) 1 +planner__cardinality__selectivity tdatetime singular 1 2001-01-01 00:00:00 1 +planner__cardinality__selectivity tdatetime singular 1 2001-01-02 00:00:00 1 +planner__cardinality__selectivity tdecimal a 0 1.00000000000000000000 1 +planner__cardinality__selectivity tdecimal a 0 2.00000000000000000000 1 +planner__cardinality__selectivity tdecimal b 0 1.00000000000000000000 1 +planner__cardinality__selectivity tdecimal b 0 2.00000000000000000000 1 +planner__cardinality__selectivity tdecimal c 0 1.00000000000000000000 1 +planner__cardinality__selectivity tdecimal c 0 2.00000000000000000000 1 +planner__cardinality__selectivity tdecimal multi 1 (1.00000000000000000000, 1.00000000000000000000) 1 +planner__cardinality__selectivity tdecimal multi 1 (2.00000000000000000000, 2.00000000000000000000) 1 +planner__cardinality__selectivity tdecimal singular 1 1.00000000000000000000 1 +planner__cardinality__selectivity tdecimal singular 1 2.00000000000000000000 1 +planner__cardinality__selectivity tdouble a 0 1 1 +planner__cardinality__selectivity tdouble a 0 2 1 +planner__cardinality__selectivity tdouble b 0 1 1 +planner__cardinality__selectivity tdouble b 0 2 1 +planner__cardinality__selectivity tdouble c 0 1 1 +planner__cardinality__selectivity tdouble c 0 2 1 +planner__cardinality__selectivity tdouble multi 1 (1, 1) 1 +planner__cardinality__selectivity tdouble multi 1 (2, 2) 1 +planner__cardinality__selectivity tdouble singular 1 1 1 +planner__cardinality__selectivity tdouble singular 1 2 1 +planner__cardinality__selectivity tint a 0 1 1 +planner__cardinality__selectivity tint a 0 2 1 +planner__cardinality__selectivity tint b 0 1 1 +planner__cardinality__selectivity tint b 0 2 1 +planner__cardinality__selectivity tint c 0 1 1 +planner__cardinality__selectivity tint c 0 2 1 +planner__cardinality__selectivity tint multi 1 (1, 1) 1 +planner__cardinality__selectivity tint multi 1 (2, 2) 1 +planner__cardinality__selectivity tint singular 1 1 1 +planner__cardinality__selectivity tint singular 1 2 1 +planner__cardinality__selectivity topn_after_hist a 0 2 2 +planner__cardinality__selectivity topn_after_hist a 0 7 4 +planner__cardinality__selectivity topn_after_hist idx 1 2 2 +planner__cardinality__selectivity topn_after_hist idx 1 7 4 +planner__cardinality__selectivity topn_after_hist_no_index a 0 2 2 +planner__cardinality__selectivity topn_after_hist_no_index a 0 7 4 +planner__cardinality__selectivity topn_before_hist a 0 1 4 +planner__cardinality__selectivity topn_before_hist a 0 3 2 +planner__cardinality__selectivity topn_before_hist idx 1 1 4 +planner__cardinality__selectivity topn_before_hist idx 1 3 2 +planner__cardinality__selectivity topn_before_hist_no_index a 0 1 4 +planner__cardinality__selectivity topn_before_hist_no_index a 0 3 2 +planner__cardinality__selectivity tprefix a 0 111 1 +planner__cardinality__selectivity tprefix a 0 222 1 +planner__cardinality__selectivity tprefix b 0 111 1 +planner__cardinality__selectivity tprefix b 0 222 1 +planner__cardinality__selectivity tprefix prefixa 1 11 1 +planner__cardinality__selectivity tprefix prefixa 1 22 1 +planner__cardinality__selectivity tstring a 0 1 1 +planner__cardinality__selectivity tstring a 0 2 1 +planner__cardinality__selectivity tstring b 0 1 1 +planner__cardinality__selectivity tstring b 0 2 1 +planner__cardinality__selectivity tstring c 0 1 1 +planner__cardinality__selectivity tstring c 0 2 1 +planner__cardinality__selectivity tstring multi 1 (1, 1) 1 +planner__cardinality__selectivity tstring multi 1 (2, 2) 1 +planner__cardinality__selectivity tstring singular 1 1 1 +planner__cardinality__selectivity tstring singular 1 2 1 +show stats_buckets where db_name = 'planner__cardinality__selectivity'; +Db_name Table_name Partition_name Column_name Is_index Bucket_id Count Repeats Lower_Bound Upper_Bound Ndv +planner__cardinality__selectivity ct1 PRIMARY 1 0 3 1 3 5 0 +planner__cardinality__selectivity ct1 PRIMARY 1 1 6 1 6 8 0 +planner__cardinality__selectivity ct1 a 0 0 3 1 3 5 0 +planner__cardinality__selectivity ct1 a 0 1 6 1 6 8 0 +planner__cardinality__selectivity ct1 pk 0 0 3 1 3 5 0 +planner__cardinality__selectivity ct1 pk 0 1 6 1 6 8 0 +planner__cardinality__selectivity ct2 PRIMARY 1 0 3 1 (3, 3) (5, 5) 0 +planner__cardinality__selectivity ct2 PRIMARY 1 1 6 1 (6, 6) (8, 8) 0 +planner__cardinality__selectivity ct2 a 0 0 3 1 3 5 0 +planner__cardinality__selectivity ct2 a 0 1 6 1 6 8 0 +planner__cardinality__selectivity ct2 b 0 0 3 1 3 5 0 +planner__cardinality__selectivity ct2 b 0 1 6 1 6 8 0 +planner__cardinality__selectivity ct2 c 0 0 3 1 3 5 0 +planner__cardinality__selectivity ct2 c 0 1 6 1 6 8 0 +planner__cardinality__selectivity tdatetime a 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0 +planner__cardinality__selectivity tdatetime a 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0 +planner__cardinality__selectivity tdatetime b 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0 +planner__cardinality__selectivity tdatetime b 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0 +planner__cardinality__selectivity tdatetime c 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0 +planner__cardinality__selectivity tdatetime c 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0 +planner__cardinality__selectivity tdatetime multi 1 0 1 1 (2001-01-03 00:00:00, 2001-01-03 00:00:00) (2001-01-03 00:00:00, 2001-01-03 00:00:00) 0 +planner__cardinality__selectivity tdatetime multi 1 1 2 1 (2001-01-04 00:00:00, 2001-01-04 00:00:00) (2001-01-04 00:00:00, 2001-01-04 00:00:00) 0 +planner__cardinality__selectivity tdatetime singular 1 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0 +planner__cardinality__selectivity tdatetime singular 1 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0 +planner__cardinality__selectivity tdecimal a 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0 +planner__cardinality__selectivity tdecimal a 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0 +planner__cardinality__selectivity tdecimal b 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0 +planner__cardinality__selectivity tdecimal b 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0 +planner__cardinality__selectivity tdecimal c 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0 +planner__cardinality__selectivity tdecimal c 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0 +planner__cardinality__selectivity tdecimal multi 1 0 3 1 (3.00000000000000000000, 3.00000000000000000000) (5.00000000000000000000, 5.00000000000000000000) 0 +planner__cardinality__selectivity tdecimal multi 1 1 6 1 (6.00000000000000000000, 6.00000000000000000000) (8.00000000000000000000, 8.00000000000000000000) 0 +planner__cardinality__selectivity tdecimal singular 1 0 3 1 3.00000000000000000000 5.00000000000000000000 0 +planner__cardinality__selectivity tdecimal singular 1 1 6 1 6.00000000000000000000 8.00000000000000000000 0 +planner__cardinality__selectivity tdouble a 0 0 3 1 3 5 0 +planner__cardinality__selectivity tdouble a 0 1 6 1 6 8 0 +planner__cardinality__selectivity tdouble b 0 0 3 1 3 5 0 +planner__cardinality__selectivity tdouble b 0 1 6 1 6 8 0 +planner__cardinality__selectivity tdouble c 0 0 3 1 3 5 0 +planner__cardinality__selectivity tdouble c 0 1 6 1 6 8 0 +planner__cardinality__selectivity tdouble multi 1 0 3 1 (3, 3) (5, 5) 0 +planner__cardinality__selectivity tdouble multi 1 1 6 1 (6, 6) (8, 8) 0 +planner__cardinality__selectivity tdouble singular 1 0 3 1 3 5 0 +planner__cardinality__selectivity tdouble singular 1 1 6 1 6 8 0 +planner__cardinality__selectivity tint a 0 0 3 1 3 5 0 +planner__cardinality__selectivity tint a 0 1 6 1 6 8 0 +planner__cardinality__selectivity tint b 0 0 3 1 3 5 0 +planner__cardinality__selectivity tint b 0 1 6 1 6 8 0 +planner__cardinality__selectivity tint c 0 0 3 1 3 5 0 +planner__cardinality__selectivity tint c 0 1 6 1 6 8 0 +planner__cardinality__selectivity tint multi 1 0 3 1 (3, 3) (5, 5) 0 +planner__cardinality__selectivity tint multi 1 1 6 1 (6, 6) (8, 8) 0 +planner__cardinality__selectivity tint singular 1 0 3 1 3 5 0 +planner__cardinality__selectivity tint singular 1 1 6 1 6 8 0 +planner__cardinality__selectivity topn_after_hist a 0 0 2 1 3 4 0 +planner__cardinality__selectivity topn_after_hist a 0 1 3 1 5 5 0 +planner__cardinality__selectivity topn_after_hist idx 1 0 2 1 3 4 0 +planner__cardinality__selectivity topn_after_hist idx 1 1 3 1 5 5 0 +planner__cardinality__selectivity topn_after_hist_no_index a 0 0 2 1 3 4 0 +planner__cardinality__selectivity topn_after_hist_no_index a 0 1 3 1 5 5 0 +planner__cardinality__selectivity topn_before_hist a 0 0 2 1 4 5 0 +planner__cardinality__selectivity topn_before_hist a 0 1 3 1 6 6 0 +planner__cardinality__selectivity topn_before_hist idx 1 0 2 1 4 5 0 +planner__cardinality__selectivity topn_before_hist idx 1 1 3 1 6 6 0 +planner__cardinality__selectivity topn_before_hist_no_index a 0 0 2 1 4 5 0 +planner__cardinality__selectivity topn_before_hist_no_index a 0 1 3 1 6 6 0 +planner__cardinality__selectivity tprefix a 0 0 2 1 333 444 0 +planner__cardinality__selectivity tprefix a 0 1 4 1 555 666 0 +planner__cardinality__selectivity tprefix b 0 0 2 1 333 444 0 +planner__cardinality__selectivity tprefix b 0 1 4 1 555 666 0 +planner__cardinality__selectivity tprefix prefixa 1 0 2 1 33 44 0 +planner__cardinality__selectivity tprefix prefixa 1 1 4 1 55 66 0 +planner__cardinality__selectivity tstring a 0 0 3 1 3 5 0 +planner__cardinality__selectivity tstring a 0 1 6 1 6 8 0 +planner__cardinality__selectivity tstring b 0 0 3 1 3 5 0 +planner__cardinality__selectivity tstring b 0 1 6 1 6 8 0 +planner__cardinality__selectivity tstring c 0 0 3 1 3 5 0 +planner__cardinality__selectivity tstring c 0 1 6 1 6 8 0 +planner__cardinality__selectivity tstring multi 1 0 3 1 (3, 3) (5, 5) 0 +planner__cardinality__selectivity tstring multi 1 1 6 1 (6, 6) (8, 8) 0 +planner__cardinality__selectivity tstring singular 1 0 3 1 3 5 0 +planner__cardinality__selectivity tstring singular 1 1 6 1 6 8 0 +explain select * from topn_before_hist where a = 1; +id estRows task access object operator info +IndexReader_6 4.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 4.00 cop[tikv] table:topn_before_hist, index:idx(a) range:[1,1], keep order:false +explain select * from topn_before_hist where a = 2; +id estRows task access object operator info +IndexReader_6 1.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 1.00 cop[tikv] table:topn_before_hist, index:idx(a) range:[2,2], keep order:false +explain select * from topn_after_hist where a = 7; +id estRows task access object operator info +IndexReader_6 4.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 4.00 cop[tikv] table:topn_after_hist, index:idx(a) range:[7,7], keep order:false +explain select * from topn_after_hist where a = 6; +id estRows task access object operator info +IndexReader_6 1.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 1.00 cop[tikv] table:topn_after_hist, index:idx(a) range:[6,6], keep order:false +explain select * from topn_after_hist_no_index where a = 7; +id estRows task access object operator info +TableReader_7 4.00 root data:Selection_6 +└─Selection_6 4.00 cop[tikv] eq(planner__cardinality__selectivity.topn_after_hist_no_index.a, 7) + └─TableFullScan_5 9.00 cop[tikv] table:topn_after_hist_no_index keep order:false +explain select * from topn_after_hist_no_index where a = 6; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.topn_after_hist_no_index.a, 6) + └─TableFullScan_5 9.00 cop[tikv] table:topn_after_hist_no_index keep order:false +explain select * from topn_before_hist_no_index where a = 1; +id estRows task access object operator info +TableReader_7 4.00 root data:Selection_6 +└─Selection_6 4.00 cop[tikv] eq(planner__cardinality__selectivity.topn_before_hist_no_index.a, 1) + └─TableFullScan_5 9.00 cop[tikv] table:topn_before_hist_no_index keep order:false +explain select * from topn_before_hist_no_index where a = 2; +id estRows task access object operator info +TableReader_7 1.00 root data:Selection_6 +└─Selection_6 1.00 cop[tikv] eq(planner__cardinality__selectivity.topn_before_hist_no_index.a, 2) + └─TableFullScan_5 9.00 cop[tikv] table:topn_before_hist_no_index keep order:false +drop table if exists t; +create table t(a char(10), b int, key idx(a, b)); +insert into t values ('cn', 0); +insert into t values ('cn', 1); +insert into t values ('cn', 2); +insert into t values ('cn', 3); +insert into t values ('cn', 4); +insert into t values ('cn', 5); +insert into t values ('cn', 6); +insert into t values ('cn', 7); +insert into t values ('cn', 8); +insert into t values ('cn', 9); +insert into t values ('cn', 10); +insert into t values ('cn', 11); +insert into t values ('cn', 12); +insert into t values ('cn', 13); +insert into t values ('cn', 14); +insert into t values ('cn', 15); +insert into t values ('cn', 16); +insert into t values ('cn', 17); +insert into t values ('cn', 18); +insert into t values ('cn', 19); +insert into t values ('cn', 20); +insert into t values ('cn', 21); +insert into t values ('cn', 22); +insert into t values ('cn', 23); +insert into t values ('cn', 24); +insert into t values ('cn', 25); +insert into t values ('cn', 26); +insert into t values ('cn', 27); +insert into t values ('cn', 28); +insert into t values ('cn', 29); +insert into t values ('cn', 30); +insert into t values ('cn', 31); +insert into t values ('cn', 32); +insert into t values ('cn', 33); +insert into t values ('cn', 34); +insert into t values ('cn', 35); +insert into t values ('cn', 36); +insert into t values ('cn', 37); +insert into t values ('cn', 38); +insert into t values ('cn', 39); +insert into t values ('cn', 40); +insert into t values ('cn', 41); +insert into t values ('cn', 42); +insert into t values ('cn', 43); +insert into t values ('cn', 44); +insert into t values ('cn', 45); +insert into t values ('cn', 46); +insert into t values ('cn', 47); +insert into t values ('cn', 48); +insert into t values ('cn', 49); +insert into t values ('cn', 50); +insert into t values ('cn', 51); +insert into t values ('cn', 52); +insert into t values ('cn', 53); +insert into t values ('cn', 54); +insert into t values ('cn', 55); +insert into t values ('cn', 56); +insert into t values ('cn', 57); +insert into t values ('cn', 58); +insert into t values ('cn', 59); +insert into t values ('cn', 60); +insert into t values ('cn', 61); +insert into t values ('cn', 62); +insert into t values ('cn', 63); +insert into t values ('cn', 64); +insert into t values ('cn', 65); +insert into t values ('cn', 66); +insert into t values ('cn', 67); +insert into t values ('cn', 68); +insert into t values ('cn', 69); +insert into t values ('cn', 70); +insert into t values ('cn', 71); +insert into t values ('cn', 72); +insert into t values ('cn', 73); +insert into t values ('cn', 74); +insert into t values ('cn', 75); +insert into t values ('cn', 76); +insert into t values ('cn', 77); +insert into t values ('cn', 78); +insert into t values ('cn', 79); +insert into t values ('cn', 80); +insert into t values ('cn', 81); +insert into t values ('cn', 82); +insert into t values ('cn', 83); +insert into t values ('cn', 84); +insert into t values ('cn', 85); +insert into t values ('cn', 86); +insert into t values ('cn', 87); +insert into t values ('cn', 88); +insert into t values ('cn', 89); +insert into t values ('cn', 90); +insert into t values ('cn', 91); +insert into t values ('cn', 92); +insert into t values ('cn', 93); +insert into t values ('cn', 94); +insert into t values ('cn', 95); +insert into t values ('cn', 96); +insert into t values ('cn', 97); +insert into t values ('cn', 98); +insert into t values ('cn', 99); +insert into t values ('cn', 100); +insert into t values ('cn', 101); +insert into t values ('cn', 102); +insert into t values ('cn', 103); +insert into t values ('cn', 104); +insert into t values ('cn', 105); +insert into t values ('cn', 106); +insert into t values ('cn', 107); +insert into t values ('cn', 108); +insert into t values ('cn', 109); +insert into t values ('cn', 110); +insert into t values ('cn', 111); +insert into t values ('cn', 112); +insert into t values ('cn', 113); +insert into t values ('cn', 114); +insert into t values ('cn', 115); +insert into t values ('cn', 116); +insert into t values ('cn', 117); +insert into t values ('cn', 118); +insert into t values ('cn', 119); +insert into t values ('cn', 120); +insert into t values ('cn', 121); +insert into t values ('cn', 122); +insert into t values ('cn', 123); +insert into t values ('cn', 124); +insert into t values ('cn', 125); +insert into t values ('cn', 126); +insert into t values ('cn', 127); +insert into t values ('cn', 128); +insert into t values ('cn', 129); +insert into t values ('cn', 130); +insert into t values ('cn', 131); +insert into t values ('cn', 132); +insert into t values ('cn', 133); +insert into t values ('cn', 134); +insert into t values ('cn', 135); +insert into t values ('cn', 136); +insert into t values ('cn', 137); +insert into t values ('cn', 138); +insert into t values ('cn', 139); +insert into t values ('cn', 140); +insert into t values ('cn', 141); +insert into t values ('cn', 142); +insert into t values ('cn', 143); +insert into t values ('cn', 144); +insert into t values ('cn', 145); +insert into t values ('cn', 146); +insert into t values ('cn', 147); +insert into t values ('cn', 148); +insert into t values ('cn', 149); +insert into t values ('cn', 150); +insert into t values ('cn', 151); +insert into t values ('cn', 152); +insert into t values ('cn', 153); +insert into t values ('cn', 154); +insert into t values ('cn', 155); +insert into t values ('cn', 156); +insert into t values ('cn', 157); +insert into t values ('cn', 158); +insert into t values ('cn', 159); +insert into t values ('cn', 160); +insert into t values ('cn', 161); +insert into t values ('cn', 162); +insert into t values ('cn', 163); +insert into t values ('cn', 164); +insert into t values ('cn', 165); +insert into t values ('cn', 166); +insert into t values ('cn', 167); +insert into t values ('cn', 168); +insert into t values ('cn', 169); +insert into t values ('cn', 170); +insert into t values ('cn', 171); +insert into t values ('cn', 172); +insert into t values ('cn', 173); +insert into t values ('cn', 174); +insert into t values ('cn', 175); +insert into t values ('cn', 176); +insert into t values ('cn', 177); +insert into t values ('cn', 178); +insert into t values ('cn', 179); +insert into t values ('cn', 180); +insert into t values ('cn', 181); +insert into t values ('cn', 182); +insert into t values ('cn', 183); +insert into t values ('cn', 184); +insert into t values ('cn', 185); +insert into t values ('cn', 186); +insert into t values ('cn', 187); +insert into t values ('cn', 188); +insert into t values ('cn', 189); +insert into t values ('cn', 190); +insert into t values ('cn', 191); +insert into t values ('cn', 192); +insert into t values ('cn', 193); +insert into t values ('cn', 194); +insert into t values ('cn', 195); +insert into t values ('cn', 196); +insert into t values ('cn', 197); +insert into t values ('cn', 198); +insert into t values ('cn', 199); +insert into t values ('cn', 200); +insert into t values ('cn', 201); +insert into t values ('cn', 202); +insert into t values ('cn', 203); +insert into t values ('cn', 204); +insert into t values ('cn', 205); +insert into t values ('cn', 206); +insert into t values ('cn', 207); +insert into t values ('cn', 208); +insert into t values ('cn', 209); +insert into t values ('cn', 210); +insert into t values ('cn', 211); +insert into t values ('cn', 212); +insert into t values ('cn', 213); +insert into t values ('cn', 214); +insert into t values ('cn', 215); +insert into t values ('cn', 216); +insert into t values ('cn', 217); +insert into t values ('cn', 218); +insert into t values ('cn', 219); +insert into t values ('cn', 220); +insert into t values ('cn', 221); +insert into t values ('cn', 222); +insert into t values ('cn', 223); +insert into t values ('cn', 224); +insert into t values ('cn', 225); +insert into t values ('cn', 226); +insert into t values ('cn', 227); +insert into t values ('cn', 228); +insert into t values ('cn', 229); +insert into t values ('cn', 230); +insert into t values ('cn', 231); +insert into t values ('cn', 232); +insert into t values ('cn', 233); +insert into t values ('cn', 234); +insert into t values ('cn', 235); +insert into t values ('cn', 236); +insert into t values ('cn', 237); +insert into t values ('cn', 238); +insert into t values ('cn', 239); +insert into t values ('cn', 240); +insert into t values ('cn', 241); +insert into t values ('cn', 242); +insert into t values ('cn', 243); +insert into t values ('cn', 244); +insert into t values ('cn', 245); +insert into t values ('cn', 246); +insert into t values ('cn', 247); +insert into t values ('cn', 248); +insert into t values ('cn', 249); +insert into t values ('cn', 250); +insert into t values ('cn', 251); +insert into t values ('cn', 252); +insert into t values ('cn', 253); +insert into t values ('cn', 254); +insert into t values ('cn', 255); +insert into t values ('cn', 256); +insert into t values ('cn', 257); +insert into t values ('cn', 258); +insert into t values ('cn', 259); +insert into t values ('cn', 260); +insert into t values ('cn', 261); +insert into t values ('cn', 262); +insert into t values ('cn', 263); +insert into t values ('cn', 264); +insert into t values ('cn', 265); +insert into t values ('cn', 266); +insert into t values ('cn', 267); +insert into t values ('cn', 268); +insert into t values ('cn', 269); +insert into t values ('cn', 270); +insert into t values ('cn', 271); +insert into t values ('cn', 272); +insert into t values ('cn', 273); +insert into t values ('cn', 274); +insert into t values ('cn', 275); +insert into t values ('cn', 276); +insert into t values ('cn', 277); +insert into t values ('cn', 278); +insert into t values ('cn', 279); +insert into t values ('cn', 280); +insert into t values ('cn', 281); +insert into t values ('cn', 282); +insert into t values ('cn', 283); +insert into t values ('cn', 284); +insert into t values ('cn', 285); +insert into t values ('cn', 286); +insert into t values ('cn', 287); +insert into t values ('cn', 288); +insert into t values ('cn', 289); +insert into t values ('cn', 290); +insert into t values ('cn', 291); +insert into t values ('cn', 292); +insert into t values ('cn', 293); +insert into t values ('cn', 294); +insert into t values ('cn', 295); +insert into t values ('cn', 296); +insert into t values ('cn', 297); +insert into t values ('cn', 298); +insert into t values ('cn', 299); +insert into t values ('cn', 300); +insert into t values ('cn', 301); +insert into t values ('cn', 302); +insert into t values ('cn', 303); +insert into t values ('cn', 304); +insert into t values ('cn', 305); +insert into t values ('cn', 306); +insert into t values ('cn', 307); +insert into t values ('cn', 308); +insert into t values ('cn', 309); +insert into t values ('cn', 310); +insert into t values ('cn', 311); +insert into t values ('cn', 312); +insert into t values ('cn', 313); +insert into t values ('cn', 314); +insert into t values ('cn', 315); +insert into t values ('cn', 316); +insert into t values ('cn', 317); +insert into t values ('cn', 318); +insert into t values ('cn', 319); +insert into t values ('cn', 320); +insert into t values ('cn', 321); +insert into t values ('cn', 322); +insert into t values ('cn', 323); +insert into t values ('cn', 324); +insert into t values ('cn', 325); +insert into t values ('cn', 326); +insert into t values ('cn', 327); +insert into t values ('cn', 328); +insert into t values ('cn', 329); +insert into t values ('cn', 330); +insert into t values ('cn', 331); +insert into t values ('cn', 332); +insert into t values ('cn', 333); +insert into t values ('cn', 334); +insert into t values ('cn', 335); +insert into t values ('cn', 336); +insert into t values ('cn', 337); +insert into t values ('cn', 338); +insert into t values ('cn', 339); +insert into t values ('cn', 340); +insert into t values ('cn', 341); +insert into t values ('cn', 342); +insert into t values ('cn', 343); +insert into t values ('cn', 344); +insert into t values ('cn', 345); +insert into t values ('cn', 346); +insert into t values ('cn', 347); +insert into t values ('cn', 348); +insert into t values ('cn', 349); +insert into t values ('cn', 350); +insert into t values ('cn', 351); +insert into t values ('cn', 352); +insert into t values ('cn', 353); +insert into t values ('cn', 354); +insert into t values ('cn', 355); +insert into t values ('cn', 356); +insert into t values ('cn', 357); +insert into t values ('cn', 358); +insert into t values ('cn', 359); +insert into t values ('cn', 360); +insert into t values ('cn', 361); +insert into t values ('cn', 362); +insert into t values ('cn', 363); +insert into t values ('cn', 364); +insert into t values ('cn', 365); +insert into t values ('cn', 366); +insert into t values ('cn', 367); +insert into t values ('cn', 368); +insert into t values ('cn', 369); +insert into t values ('cn', 370); +insert into t values ('cn', 371); +insert into t values ('cn', 372); +insert into t values ('cn', 373); +insert into t values ('cn', 374); +insert into t values ('cn', 375); +insert into t values ('cn', 376); +insert into t values ('cn', 377); +insert into t values ('cn', 378); +insert into t values ('cn', 379); +insert into t values ('cn', 380); +insert into t values ('cn', 381); +insert into t values ('cn', 382); +insert into t values ('cn', 383); +insert into t values ('cn', 384); +insert into t values ('cn', 385); +insert into t values ('cn', 386); +insert into t values ('cn', 387); +insert into t values ('cn', 388); +insert into t values ('cn', 389); +insert into t values ('cn', 390); +insert into t values ('cn', 391); +insert into t values ('cn', 392); +insert into t values ('cn', 393); +insert into t values ('cn', 394); +insert into t values ('cn', 395); +insert into t values ('cn', 396); +insert into t values ('cn', 397); +insert into t values ('cn', 398); +insert into t values ('cn', 399); +insert into t values ('cn', 400); +insert into t values ('cn', 401); +insert into t values ('cn', 402); +insert into t values ('cn', 403); +insert into t values ('cn', 404); +insert into t values ('cn', 405); +insert into t values ('cn', 406); +insert into t values ('cn', 407); +insert into t values ('cn', 408); +insert into t values ('cn', 409); +insert into t values ('cn', 410); +insert into t values ('cn', 411); +insert into t values ('cn', 412); +insert into t values ('cn', 413); +insert into t values ('cn', 414); +insert into t values ('cn', 415); +insert into t values ('cn', 416); +insert into t values ('cn', 417); +insert into t values ('cn', 418); +insert into t values ('cn', 419); +insert into t values ('cn', 420); +insert into t values ('cn', 421); +insert into t values ('cn', 422); +insert into t values ('cn', 423); +insert into t values ('cn', 424); +insert into t values ('cn', 425); +insert into t values ('cn', 426); +insert into t values ('cn', 427); +insert into t values ('cn', 428); +insert into t values ('cn', 429); +insert into t values ('cn', 430); +insert into t values ('cn', 431); +insert into t values ('cn', 432); +insert into t values ('cn', 433); +insert into t values ('cn', 434); +insert into t values ('cn', 435); +insert into t values ('cn', 436); +insert into t values ('cn', 437); +insert into t values ('cn', 438); +insert into t values ('cn', 439); +insert into t values ('cn', 440); +insert into t values ('cn', 441); +insert into t values ('cn', 442); +insert into t values ('cn', 443); +insert into t values ('cn', 444); +insert into t values ('cn', 445); +insert into t values ('cn', 446); +insert into t values ('cn', 447); +insert into t values ('cn', 448); +insert into t values ('cn', 449); +insert into t values ('cn', 450); +insert into t values ('cn', 451); +insert into t values ('cn', 452); +insert into t values ('cn', 453); +insert into t values ('cn', 454); +insert into t values ('cn', 455); +insert into t values ('cn', 456); +insert into t values ('cn', 457); +insert into t values ('cn', 458); +insert into t values ('cn', 459); +insert into t values ('cn', 460); +insert into t values ('cn', 461); +insert into t values ('cn', 462); +insert into t values ('cn', 463); +insert into t values ('cn', 464); +insert into t values ('cn', 465); +insert into t values ('cn', 466); +insert into t values ('cn', 467); +insert into t values ('cn', 468); +insert into t values ('cn', 469); +insert into t values ('cn', 470); +insert into t values ('cn', 471); +insert into t values ('cn', 472); +insert into t values ('cn', 473); +insert into t values ('cn', 474); +insert into t values ('cn', 475); +insert into t values ('cn', 476); +insert into t values ('cn', 477); +insert into t values ('cn', 478); +insert into t values ('cn', 479); +insert into t values ('cn', 480); +insert into t values ('cn', 481); +insert into t values ('cn', 482); +insert into t values ('cn', 483); +insert into t values ('cn', 484); +insert into t values ('cn', 485); +insert into t values ('cn', 486); +insert into t values ('cn', 487); +insert into t values ('cn', 488); +insert into t values ('cn', 489); +insert into t values ('cn', 490); +insert into t values ('cn', 491); +insert into t values ('cn', 492); +insert into t values ('cn', 493); +insert into t values ('cn', 494); +insert into t values ('cn', 495); +insert into t values ('cn', 496); +insert into t values ('cn', 497); +insert into t values ('cn', 498); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +analyze table t all columns; +explain select * from t where a = 'tw' and b < 0; +id estRows task access object operator info +IndexReader_6 1.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 1.00 cop[tikv] table:t, index:idx(a, b) range:["tw" -inf,"tw" 0), keep order:false +set @@tidb_opt_fix_control = '47400:on'; +explain select * from t where a = 'tw' and b < 0; +id estRows task access object operator info +IndexReader_6 0.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.00 cop[tikv] table:t, index:idx(a, b) range:["tw" -inf,"tw" 0), keep order:false +set @@tidb_opt_fix_control = '47400:off'; +drop table if exists t; +create table t(id int auto_increment, kid int, pid int, primary key(id), key(kid, pid)); +insert into t (kid, pid) values (1,2), (1,3), (1,4),(1, 11), (1, 12), (1, 13), (1, 14), (2, 2), (2, 3), (2, 4); +analyze table t all columns; +explain select * from t where kid = 1; +id estRows task access object operator info +IndexReader_6 7.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 7.00 cop[tikv] table:t, index:kid(kid, pid) range:[1,1], keep order:false +drop table if exists t; +create table t(a int, b varchar(100)); +set @@tidb_default_string_match_selectivity = 0.8; +explain format = 'brief' select * from t where a + 10 > 100; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] gt(plus(planner__cardinality__selectivity.t.a, 10), 100) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b like '%test%'; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] like(planner__cardinality__selectivity.t.b, "%test%", 92) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b not like '%test%'; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] not(like(planner__cardinality__selectivity.t.b, "%test%", 92)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b regexp '.*test.*'; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] regexp(planner__cardinality__selectivity.t.b, ".*test.*") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b not regexp '.*test.*'; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] not(istrue_with_null(regexp(planner__cardinality__selectivity.t.b, ".*test.*"))) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] gt(plus(planner__cardinality__selectivity.t.a, 10), 100), like(planner__cardinality__selectivity.t.b, "%test%", 92) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] gt(plus(planner__cardinality__selectivity.t.a, 10), 100), not(like(planner__cardinality__selectivity.t.b, "%test%", 92)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100; +id estRows task access object operator info +TableReader 9600.00 root data:Selection +└─Selection 9600.00 cop[tikv] or(like(planner__cardinality__selectivity.t.b, "%test%", 92), gt(plus(planner__cardinality__selectivity.t.a, 10), 100)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +set @@tidb_default_string_match_selectivity = 0.9; +explain format = 'brief' select * from t where a + 10 > 100; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] gt(plus(planner__cardinality__selectivity.t.a, 10), 100) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b like '%test%'; +id estRows task access object operator info +TableReader 9000.00 root data:Selection +└─Selection 9000.00 cop[tikv] like(planner__cardinality__selectivity.t.b, "%test%", 92) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b not like '%test%'; +id estRows task access object operator info +TableReader 1000.00 root data:Selection +└─Selection 1000.00 cop[tikv] not(like(planner__cardinality__selectivity.t.b, "%test%", 92)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b regexp '.*test.*'; +id estRows task access object operator info +TableReader 9000.00 root data:Selection +└─Selection 9000.00 cop[tikv] regexp(planner__cardinality__selectivity.t.b, ".*test.*") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b not regexp '.*test.*'; +id estRows task access object operator info +TableReader 1000.00 root data:Selection +└─Selection 1000.00 cop[tikv] not(istrue_with_null(regexp(planner__cardinality__selectivity.t.b, ".*test.*"))) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] gt(plus(planner__cardinality__selectivity.t.a, 10), 100), like(planner__cardinality__selectivity.t.b, "%test%", 92) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100; +id estRows task access object operator info +TableReader 1000.00 root data:Selection +└─Selection 1000.00 cop[tikv] gt(plus(planner__cardinality__selectivity.t.a, 10), 100), not(like(planner__cardinality__selectivity.t.b, "%test%", 92)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100; +id estRows task access object operator info +TableReader 9800.00 root data:Selection +└─Selection 9800.00 cop[tikv] or(like(planner__cardinality__selectivity.t.b, "%test%", 92), gt(plus(planner__cardinality__selectivity.t.a, 10), 100)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +set @@tidb_default_string_match_selectivity = 0.1; +explain format = 'brief' select * from t where a + 10 > 100; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] gt(plus(planner__cardinality__selectivity.t.a, 10), 100) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b like '%test%'; +id estRows task access object operator info +TableReader 1000.00 root data:Selection +└─Selection 1000.00 cop[tikv] like(planner__cardinality__selectivity.t.b, "%test%", 92) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b like '%test%' is true; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] istrue(like(planner__cardinality__selectivity.t.b, "%test%", 92)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b not like '%test%'; +id estRows task access object operator info +TableReader 9000.00 root data:Selection +└─Selection 9000.00 cop[tikv] not(like(planner__cardinality__selectivity.t.b, "%test%", 92)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b regexp '.*test.*'; +id estRows task access object operator info +TableReader 1000.00 root data:Selection +└─Selection 1000.00 cop[tikv] regexp(planner__cardinality__selectivity.t.b, ".*test.*") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b rlike '.*test.*'; +id estRows task access object operator info +TableReader 1000.00 root data:Selection +└─Selection 1000.00 cop[tikv] regexp(planner__cardinality__selectivity.t.b, ".*test.*") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b not rlike '.*test.*'; +id estRows task access object operator info +TableReader 9000.00 root data:Selection +└─Selection 9000.00 cop[tikv] not(istrue_with_null(regexp(planner__cardinality__selectivity.t.b, ".*test.*"))) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b not regexp '.*test.*'; +id estRows task access object operator info +TableReader 9000.00 root data:Selection +└─Selection 9000.00 cop[tikv] not(istrue_with_null(regexp(planner__cardinality__selectivity.t.b, ".*test.*"))) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100; +id estRows task access object operator info +TableReader 1000.00 root data:Selection +└─Selection 1000.00 cop[tikv] gt(plus(planner__cardinality__selectivity.t.a, 10), 100), like(planner__cardinality__selectivity.t.b, "%test%", 92) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] gt(plus(planner__cardinality__selectivity.t.a, 10), 100), not(like(planner__cardinality__selectivity.t.b, "%test%", 92)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100; +id estRows task access object operator info +TableReader 8200.00 root data:Selection +└─Selection 8200.00 cop[tikv] or(like(planner__cardinality__selectivity.t.b, "%test%", 92), gt(plus(planner__cardinality__selectivity.t.a, 10), 100)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +set @@tidb_default_string_match_selectivity = 0; +explain format = 'brief' select * from t where a + 10 > 100; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] gt(plus(planner__cardinality__selectivity.t.a, 10), 100) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b like '%test%'; +id estRows task access object operator info +TableReader 1000.00 root data:Selection +└─Selection 1000.00 cop[tikv] like(planner__cardinality__selectivity.t.b, "%test%", 92) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b not like '%test%'; +id estRows task access object operator info +TableReader 9000.00 root data:Selection +└─Selection 9000.00 cop[tikv] not(like(planner__cardinality__selectivity.t.b, "%test%", 92)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b regexp '.*test.*'; +id estRows task access object operator info +TableReader 1000.00 root data:Selection +└─Selection 1000.00 cop[tikv] regexp(planner__cardinality__selectivity.t.b, ".*test.*") + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b not regexp '.*test.*'; +id estRows task access object operator info +TableReader 9000.00 root data:Selection +└─Selection 9000.00 cop[tikv] not(istrue_with_null(regexp(planner__cardinality__selectivity.t.b, ".*test.*"))) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100; +id estRows task access object operator info +TableReader 1000.00 root data:Selection +└─Selection 1000.00 cop[tikv] gt(plus(planner__cardinality__selectivity.t.a, 10), 100), like(planner__cardinality__selectivity.t.b, "%test%", 92) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100; +id estRows task access object operator info +TableReader 8000.00 root data:Selection +└─Selection 8000.00 cop[tikv] gt(plus(planner__cardinality__selectivity.t.a, 10), 100), not(like(planner__cardinality__selectivity.t.b, "%test%", 92)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100; +id estRows task access object operator info +TableReader 8200.00 root data:Selection +└─Selection 8200.00 cop[tikv] or(like(planner__cardinality__selectivity.t.b, "%test%", 92), gt(plus(planner__cardinality__selectivity.t.a, 10), 100)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo diff --git a/tests/integrationtest/r/planner/core/casetest/integration.result b/tests/integrationtest/r/planner/core/casetest/integration.result new file mode 100644 index 0000000000000..9946469915370 --- /dev/null +++ b/tests/integrationtest/r/planner/core/casetest/integration.result @@ -0,0 +1,1822 @@ +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int); +insert into t values(1),(2); +select count(1) from t join (select count(1) from t where false) as tmp; +count(1) +2 +select count(1) from t join (select max(a) from t where false) as tmp; +count(1) +2 +select count(1) from t join (select min(a) from t where false) as tmp; +count(1) +2 +select count(1) from t join (select sum(a) from t where false) as tmp; +count(1) +2 +select count(1) from t join (select avg(a) from t where false) as tmp; +count(1) +2 +select count(1) from t join (select count(1) from t where false group by a) as tmp; +count(1) +0 +select count(1) from t join (select max(a) from t where false group by a) as tmp; +count(1) +0 +select count(1) from t join (select min(a) from t where false group by a) as tmp; +count(1) +0 +select count(1) from t join (select sum(a) from t where false group by a) as tmp; +count(1) +0 +select count(1) from t join (select avg(a) from t where false group by a) as tmp; +count(1) +0 +SELECT avg(2) FROM(SELECT min(c) FROM t JOIN(SELECT 1 c) d ORDER BY a) e; +avg(2) +2.0000 +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a bigint, b bigint); +explain format = 'brief' select * from t t1 left join t t2 on t1.a=t2.a where from_unixtime(t2.b); +id estRows task access object operator info +Projection 9990.00 root planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.b, planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.b +└─HashJoin 9990.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] + ├─Selection(Build) 7992.00 root from_unixtime(cast(planner__core__casetest__integration.t.b, decimal(20,0) BINARY)) + │ └─TableReader 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int not null, b datetime default null); +explain format = 'brief' select * from t t1 left join t t2 on t1.a = t2.a where cast(t1.b as date) >= '2019-01-01'; +id estRows task access object operator info +HashJoin 10000.00 root left outer join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +├─TableReader(Build) 8000.00 root data:Selection +│ └─Selection 8000.00 cop[tikv] ge(cast(planner__core__casetest__integration.t.b, date BINARY), 2019-01-01 00:00:00.000000) +│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +set @@tidb_partition_prune_mode='static'; +set tidb_opt_limit_push_down_threshold=0; +drop table if exists t; +create table t(a int, b int)partition by range columns(a)(partition p0 values less than (10), partition p1 values less than(20), partition p2 values less than(30)); +insert into t values(21, 1), (22, 2), (23, 3), (24, 4), (15, 5); +analyze table t all columns; +explain format = 'brief' select * from t order by a; +id estRows task access object operator info +Sort 10005.00 root planner__core__casetest__integration.t.a +└─PartitionUnion 10005.00 root + ├─TableReader 10000.00 root data:TableFullScan + │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo + ├─TableReader 1.00 root data:TableFullScan + │ └─TableFullScan 1.00 cop[tikv] table:t, partition:p1 keep order:false + └─TableReader 4.00 root data:TableFullScan + └─TableFullScan 4.00 cop[tikv] table:t, partition:p2 keep order:false +select * from t order by a; +a b +15 5 +21 1 +22 2 +23 3 +24 4 +explain format = 'brief' select * from t order by a limit 3; +id estRows task access object operator info +TopN 3.00 root planner__core__casetest__integration.t.a, offset:0, count:3 +└─PartitionUnion 7.00 root + ├─TopN 3.00 root planner__core__casetest__integration.t.a, offset:0, count:3 + │ └─TableReader 3.00 root data:TopN + │ └─TopN 3.00 cop[tikv] planner__core__casetest__integration.t.a, offset:0, count:3 + │ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo + ├─TopN 1.00 root planner__core__casetest__integration.t.a, offset:0, count:3 + │ └─TableReader 1.00 root data:TableFullScan + │ └─TableFullScan 1.00 cop[tikv] table:t, partition:p1 keep order:false + └─TopN 3.00 root planner__core__casetest__integration.t.a, offset:0, count:3 + └─TableReader 3.00 root data:TopN + └─TopN 3.00 cop[tikv] planner__core__casetest__integration.t.a, offset:0, count:3 + └─TableFullScan 4.00 cop[tikv] table:t, partition:p2 keep order:false +select * from t order by a limit 3; +a b +15 5 +21 1 +22 2 +set tidb_opt_limit_push_down_threshold=default; +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int primary key); +set tidb_enable_clustered_index='ON'; +create table cluster_index_t(a int, b int, c int, primary key (a, b)); +explain format = 'brief' (select max(a) from t) union (select min(a) from t); +id estRows task access object operator info +HashAgg 2.00 root group by:Column#5, funcs:firstrow(Column#5)->Column#5 +└─Union 2.00 root + ├─StreamAgg 1.00 root funcs:max(planner__core__casetest__integration.t.a)->Column#2 + │ └─Limit 1.00 root offset:0, count:1 + │ └─TableReader 1.00 root data:Limit + │ └─Limit 1.00 cop[tikv] offset:0, count:1 + │ └─TableFullScan 1.00 cop[tikv] table:t keep order:true, desc, stats:pseudo + └─StreamAgg 1.00 root funcs:min(planner__core__casetest__integration.t.a)->Column#4 + └─Limit 1.00 root offset:0, count:1 + └─TableReader 1.00 root data:Limit + └─Limit 1.00 cop[tikv] offset:0, count:1 + └─TableFullScan 1.00 cop[tikv] table:t keep order:true, stats:pseudo +explain format = 'brief' select min(a), max(a) from cluster_index_t; +id estRows task access object operator info +HashJoin 1.00 root CARTESIAN inner join +├─StreamAgg(Build) 1.00 root funcs:max(planner__core__casetest__integration.cluster_index_t.a)->Column#5 +│ └─Limit 1.00 root offset:0, count:1 +│ └─TableReader 1.00 root data:Limit +│ └─Limit 1.00 cop[tikv] offset:0, count:1 +│ └─TableFullScan 1.00 cop[tikv] table:cluster_index_t keep order:true, desc, stats:pseudo +└─StreamAgg(Probe) 1.00 root funcs:min(planner__core__casetest__integration.cluster_index_t.a)->Column#4 + └─Limit 1.00 root offset:0, count:1 + └─TableReader 1.00 root data:Limit + └─Limit 1.00 cop[tikv] offset:0, count:1 + └─TableFullScan 1.00 cop[tikv] table:cluster_index_t keep order:true, stats:pseudo +explain format = 'brief' select min(b), max(b) from cluster_index_t where a = 1; +id estRows task access object operator info +HashJoin 1.00 root CARTESIAN inner join +├─StreamAgg(Build) 1.00 root funcs:max(planner__core__casetest__integration.cluster_index_t.b)->Column#5 +│ └─Limit 1.00 root offset:0, count:1 +│ └─TableReader 1.00 root data:Limit +│ └─Limit 1.00 cop[tikv] offset:0, count:1 +│ └─TableRangeScan 1.00 cop[tikv] table:cluster_index_t range:[1,1], keep order:true, desc, stats:pseudo +└─StreamAgg(Probe) 1.00 root funcs:min(planner__core__casetest__integration.cluster_index_t.b)->Column#4 + └─Limit 1.00 root offset:0, count:1 + └─TableReader 1.00 root data:Limit + └─Limit 1.00 cop[tikv] offset:0, count:1 + └─TableRangeScan 1.00 cop[tikv] table:cluster_index_t range:[1,1], keep order:true, stats:pseudo +explain format = 'brief' select min(a), max(a) from cluster_index_t where b = 1; +id estRows task access object operator info +StreamAgg 1.00 root funcs:min(Column#8)->Column#4, funcs:max(Column#9)->Column#5 +└─TableReader 1.00 root data:StreamAgg + └─StreamAgg 1.00 cop[tikv] funcs:min(planner__core__casetest__integration.cluster_index_t.a)->Column#8, funcs:max(planner__core__casetest__integration.cluster_index_t.a)->Column#9 + └─Selection 10.00 cop[tikv] eq(planner__core__casetest__integration.cluster_index_t.b, 1) + └─TableFullScan 10000.00 cop[tikv] table:cluster_index_t keep order:false, stats:pseudo +explain format = 'brief' select min(b), max(b) from cluster_index_t where b = 1; +id estRows task access object operator info +StreamAgg 1.00 root funcs:min(Column#8)->Column#4, funcs:max(Column#9)->Column#5 +└─TableReader 1.00 root data:StreamAgg + └─StreamAgg 1.00 cop[tikv] funcs:min(planner__core__casetest__integration.cluster_index_t.b)->Column#8, funcs:max(planner__core__casetest__integration.cluster_index_t.b)->Column#9 + └─Selection 10.00 cop[tikv] eq(planner__core__casetest__integration.cluster_index_t.b, 1) + └─TableFullScan 10000.00 cop[tikv] table:cluster_index_t keep order:false, stats:pseudo +set tidb_enable_clustered_index=DEFAULT; +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int, b int); +desc format = 'brief' select t1.b from t t1 where t1.b in (select t2.a from t t2 order by t1.a+t2.a limit 1); +id estRows task access object operator info +Projection 9990.00 root planner__core__casetest__integration.t.b +└─Apply 9990.00 root semi join, equal:[eq(planner__core__casetest__integration.t.b, planner__core__casetest__integration.t.a)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.b)) + │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo + └─Selection(Probe) 7992.00 root not(isnull(planner__core__casetest__integration.t.a)) + └─Projection 9990.00 root planner__core__casetest__integration.t.a + └─TopN 9990.00 root Column#7, offset:0, count:1 + └─Projection 9990.00 root planner__core__casetest__integration.t.a, plus(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)->Column#7 + └─TableReader 9990.00 root data:TopN + └─TopN 9990.00 cop[tikv] plus(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a), offset:0, count:1 + └─TableFullScan 99900000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +desc format = 'brief' select t1.a from t t1 order by (t1.b = 1 and exists (select 1 from t t2 where t1.b = t2.b)) limit 1; +id estRows task access object operator info +Projection 1.00 root planner__core__casetest__integration.t.a +└─Projection 1.00 root planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.b, Column#11 + └─TopN 1.00 root Column#13, offset:0, count:1 + └─Projection 10000.00 root planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.b, Column#11, and(eq(planner__core__casetest__integration.t.b, 1), Column#11)->Column#13 + └─HashJoin 10000.00 root left outer semi join, equal:[eq(planner__core__casetest__integration.t.b, planner__core__casetest__integration.t.b)] + ├─TableReader(Build) 10000.00 root data:TableFullScan + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +desc format = 'brief' select * from (select b+b as x from t) t1, t t2 where t1.x=t2.b order by t1.x limit 1; +id estRows task access object operator info +TopN 1.00 root Column#4, offset:0, count:1 +└─Projection 10000.00 root Column#4, planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.b + └─HashJoin 10000.00 root inner join, equal:[eq(planner__core__casetest__integration.t.b, Column#4)] + ├─Projection(Build) 8000.00 root plus(planner__core__casetest__integration.t.b, planner__core__casetest__integration.t.b)->Column#4 + │ └─TableReader 8000.00 root data:Selection + │ └─Selection 8000.00 cop[tikv] not(isnull(plus(planner__core__casetest__integration.t.b, planner__core__casetest__integration.t.b))) + │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.b)) + └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +drop table if exists t; +create table t(a int, b int); +insert into t values(1, 1), (2, 1), (3, 2), (4, 2), (5, 2); +explain select approx_percentile(a, 50) from t; +id estRows task access object operator info +HashAgg_5 1.00 root funcs:approx_percentile(planner__core__casetest__integration.t.a, 50)->Column#4 +└─TableReader_11 10000.00 root data:TableFullScan_10 + └─TableFullScan_10 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select approx_percentile(a, 50) from t; +approx_percentile(a, 50) +3 +explain select approx_percentile(a, 10) from t; +id estRows task access object operator info +HashAgg_5 1.00 root funcs:approx_percentile(planner__core__casetest__integration.t.a, 10)->Column#4 +└─TableReader_11 10000.00 root data:TableFullScan_10 + └─TableFullScan_10 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select approx_percentile(a, 10) from t; +approx_percentile(a, 10) +1 +explain select approx_percentile(a, 10+70) from t; +id estRows task access object operator info +HashAgg_5 1.00 root funcs:approx_percentile(planner__core__casetest__integration.t.a, 80)->Column#4 +└─TableReader_11 10000.00 root data:TableFullScan_10 + └─TableFullScan_10 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select approx_percentile(a, 10+70) from t; +approx_percentile(a, 10+70) +4 +explain select approx_percentile(a, 10*10) from t; +id estRows task access object operator info +HashAgg_5 1.00 root funcs:approx_percentile(planner__core__casetest__integration.t.a, 100)->Column#4 +└─TableReader_11 10000.00 root data:TableFullScan_10 + └─TableFullScan_10 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select approx_percentile(a, 10*10) from t; +approx_percentile(a, 10*10) +5 +explain select approx_percentile(a, 50) from t group by b order by b; +id estRows task access object operator info +Projection_6 8000.00 root Column#4->Column#5 +└─Sort_7 8000.00 root planner__core__casetest__integration.t.b + └─HashAgg_9 8000.00 root group by:planner__core__casetest__integration.t.b, funcs:approx_percentile(planner__core__casetest__integration.t.a, 50)->Column#4, funcs:firstrow(planner__core__casetest__integration.t.b)->planner__core__casetest__integration.t.b + └─TableReader_13 10000.00 root data:TableFullScan_12 + └─TableFullScan_12 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select approx_percentile(a, 50) from t group by b order by b; +approx_percentile(a, 50) +1 +4 +drop table if exists t; +create table t(a int); +insert into t values(1),(1),(2); +explain format = 'brief' select /*+ stream_agg() */ count(*) c from t group by a order by c limit 1; +id estRows task access object operator info +TopN 1.00 root Column#3, offset:0, count:1 +└─StreamAgg 8000.00 root group by:planner__core__casetest__integration.t.a, funcs:count(1)->Column#3 + └─Sort 10000.00 root planner__core__casetest__integration.t.a + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select /*+ stream_agg() */ count(*) c from t group by a order by c limit 1; +c +1 +explain format = 'brief' select /*+ stream_agg() */ count(*) c from t group by a order by c; +id estRows task access object operator info +Sort 8000.00 root Column#3 +└─StreamAgg 8000.00 root group by:planner__core__casetest__integration.t.a, funcs:count(1)->Column#3 + └─Sort 10000.00 root planner__core__casetest__integration.t.a + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select /*+ stream_agg() */ count(*) c from t group by a order by c; +c +1 +2 +explain format = 'brief' select /*+ stream_agg() */ count(*) c from t group by a order by a limit 1; +id estRows task access object operator info +Projection 1.00 root Column#3->Column#4 +└─Limit 1.00 root offset:0, count:1 + └─StreamAgg 1.00 root group by:planner__core__casetest__integration.t.a, funcs:count(1)->Column#3, funcs:firstrow(planner__core__casetest__integration.t.a)->planner__core__casetest__integration.t.a + └─Sort 1.25 root planner__core__casetest__integration.t.a + └─TableReader 1.25 root data:TableFullScan + └─TableFullScan 1.25 cop[tikv] table:t keep order:false, stats:pseudo +select /*+ stream_agg() */ count(*) c from t group by a order by a limit 1; +c +2 +explain format = 'brief' select /*+ stream_agg() */ count(*) c from t group by a order by a; +id estRows task access object operator info +Projection 8000.00 root Column#3->Column#4 +└─StreamAgg 8000.00 root group by:planner__core__casetest__integration.t.a, funcs:count(1)->Column#3, funcs:firstrow(planner__core__casetest__integration.t.a)->planner__core__casetest__integration.t.a + └─Sort 10000.00 root planner__core__casetest__integration.t.a + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select /*+ stream_agg() */ count(*) c from t group by a order by a; +c +2 +1 +drop table if exists t; +drop table if exists s; +create table t(a int, b int); +create table s(a int, b int, index(a)); +insert into t values(1,1),(1,2),(2,2); +insert into s values(1,1),(2,2),(2,1); +explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.b; +id estRows task access object operator info +IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t.a, inner key:planner__core__casetest__integration.s.a, equal cond:eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.a), eq(planner__core__casetest__integration.t.b, planner__core__casetest__integration.s.b) +├─TableReader(Build) 9980.01 root data:Selection +│ └─Selection 9980.01 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)), not(isnull(planner__core__casetest__integration.t.b)) +│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +└─IndexLookUp(Probe) 12475.01 root + ├─Selection(Build) 12487.50 cop[tikv] not(isnull(planner__core__casetest__integration.s.a)) + │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(planner__core__casetest__integration.s.a, planner__core__casetest__integration.t.a)], keep order:false, stats:pseudo + └─Selection(Probe) 12475.01 cop[tikv] not(isnull(planner__core__casetest__integration.s.b)) + └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo +explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.a; +id estRows task access object operator info +IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t.a, inner key:planner__core__casetest__integration.s.a, equal cond:eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.a), eq(planner__core__casetest__integration.t.b, planner__core__casetest__integration.s.a) +├─TableReader(Build) 9980.01 root data:Selection +│ └─Selection 9980.01 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)), not(isnull(planner__core__casetest__integration.t.b)) +│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +└─IndexLookUp(Probe) 12475.01 root + ├─Selection(Build) 12475.01 cop[tikv] not(isnull(planner__core__casetest__integration.s.a)) + │ └─IndexRangeScan 12487.50 cop[tikv] table:s, index:a(a) range: decided by [eq(planner__core__casetest__integration.s.a, planner__core__casetest__integration.t.a)], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:s keep order:false, stats:pseudo +explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.a = s.b; +id estRows task access object operator info +Projection 12475.01 root planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.b, planner__core__casetest__integration.s.a, planner__core__casetest__integration.s.b +└─IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t.a, inner key:planner__core__casetest__integration.s.a, equal cond:eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.a), eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.b) + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)) + │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo + └─IndexLookUp(Probe) 12475.01 root + ├─Selection(Build) 12487.50 cop[tikv] not(isnull(planner__core__casetest__integration.s.a)) + │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(planner__core__casetest__integration.s.a, planner__core__casetest__integration.t.a)], keep order:false, stats:pseudo + └─Selection(Probe) 12475.01 cop[tikv] not(isnull(planner__core__casetest__integration.s.b)) + └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo +explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.b; +id estRows task access object operator info +IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t.a, inner key:planner__core__casetest__integration.s.a, equal cond:eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.a), eq(planner__core__casetest__integration.t.b, planner__core__casetest__integration.s.b) +├─TableReader(Build) 9980.01 root data:Selection +│ └─Selection 9980.01 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)), not(isnull(planner__core__casetest__integration.t.b)) +│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +└─IndexLookUp(Probe) 12475.01 root + ├─Selection(Build) 12487.50 cop[tikv] not(isnull(planner__core__casetest__integration.s.a)) + │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(planner__core__casetest__integration.s.a, planner__core__casetest__integration.t.a)], keep order:false, stats:pseudo + └─Selection(Probe) 12475.01 cop[tikv] not(isnull(planner__core__casetest__integration.s.b)) + └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo +explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.a; +id estRows task access object operator info +IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t.a, inner key:planner__core__casetest__integration.s.a, equal cond:eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.a), eq(planner__core__casetest__integration.t.b, planner__core__casetest__integration.s.a) +├─TableReader(Build) 9980.01 root data:Selection +│ └─Selection 9980.01 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)), not(isnull(planner__core__casetest__integration.t.b)) +│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +└─IndexLookUp(Probe) 12475.01 root + ├─Selection(Build) 12475.01 cop[tikv] not(isnull(planner__core__casetest__integration.s.a)) + │ └─IndexRangeScan 12487.50 cop[tikv] table:s, index:a(a) range: decided by [eq(planner__core__casetest__integration.s.a, planner__core__casetest__integration.t.a)], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:s keep order:false, stats:pseudo +explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.a = s.b; +id estRows task access object operator info +Projection 12475.01 root planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.b, planner__core__casetest__integration.s.a, planner__core__casetest__integration.s.b +└─IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t.a, inner key:planner__core__casetest__integration.s.a, equal cond:eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.a), eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.s.b) + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.a)) + │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo + └─IndexLookUp(Probe) 12475.01 root + ├─Selection(Build) 12487.50 cop[tikv] not(isnull(planner__core__casetest__integration.s.a)) + │ └─IndexRangeScan 12500.00 cop[tikv] table:s, index:a(a) range: decided by [eq(planner__core__casetest__integration.s.a, planner__core__casetest__integration.t.a)], keep order:false, stats:pseudo + └─Selection(Probe) 12475.01 cop[tikv] not(isnull(planner__core__casetest__integration.s.b)) + └─TableRowIDScan 12487.50 cop[tikv] table:s keep order:false, stats:pseudo +drop table if exists t; +create table t(a int, b int); +insert into t values(1, 2), (3, 4); +explain format = 'brief' select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t; +id estRows task access object operator info +HashJoin 10000.00 root CARTESIAN left outer semi join, other cond:eq(2, planner__core__casetest__integration.t.b) +├─TableReader(Build) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t; +(2) in (select b from t) +1 +1 +drop table if exists t1; +create table t1 (c1 int primary key, c2 int, c3 int, index c2 (c2)); +select count(1) from (select count(1) from (select * from t1 where c3 = 100) k) k2; +count(1) +1 +set tidb_cost_model_version=2; +drop table if exists t1,t2,t3; +create table t1 (pk char(32) primary key nonclustered, col1 char(32), col2 varchar(40), col3 char(32), key (col1), key (col3), key (col2,col3), key (col1,col3)); +create table t2 (pk char(32) primary key nonclustered, col1 varchar(100)); +create table t3 (pk char(32) primary key nonclustered, keycol varchar(100), pad1 tinyint(1) default null, pad2 varchar(40), key (keycol,pad1,pad2)); +explain format = 'brief' SELECT t1.pk FROM t1 INNER JOIN t2 ON t1.col1 = t2.pk INNER JOIN t3 ON t1.col3 = t3.pk WHERE t2.col1 IN ('a' , 'b') AND t3.keycol = 'c' AND t1.col2 = 'a' AND t1.col1 != 'abcdef' AND t1.col1 != 'aaaaaa'; +id estRows task access object operator info +IndexHashJoin 13.81 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t1.col1, inner key:planner__core__casetest__integration.t2.pk, equal cond:eq(planner__core__casetest__integration.t1.col1, planner__core__casetest__integration.t2.pk) +├─IndexHashJoin(Build) 12.50 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t3.pk, inner key:planner__core__casetest__integration.t1.col3, equal cond:eq(planner__core__casetest__integration.t3.pk, planner__core__casetest__integration.t1.col3) +│ ├─IndexLookUp(Build) 10.00 root +│ │ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t3, index:keycol(keycol, pad1, pad2) range:["c","c"], keep order:false, stats:pseudo +│ │ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t3 keep order:false, stats:pseudo +│ └─IndexLookUp(Probe) 12.50 root +│ ├─Selection(Build) 18.10 cop[tikv] not(isnull(planner__core__casetest__integration.t1.col3)) +│ │ └─IndexRangeScan 18.12 cop[tikv] table:t1, index:col2(col2, col3) range: decided by [eq(planner__core__casetest__integration.t1.col3, planner__core__casetest__integration.t3.pk) eq(planner__core__casetest__integration.t1.col2, a)], keep order:false, stats:pseudo +│ └─Selection(Probe) 12.50 cop[tikv] ne(planner__core__casetest__integration.t1.col1, "aaaaaa"), ne(planner__core__casetest__integration.t1.col1, "abcdef"), not(isnull(planner__core__casetest__integration.t1.col1)) +│ └─TableRowIDScan 18.10 cop[tikv] table:t1 keep order:false, stats:pseudo +└─IndexLookUp(Probe) 12.50 root + ├─Selection(Build) 12.50 cop[tikv] ne(planner__core__casetest__integration.t2.pk, "aaaaaa"), ne(planner__core__casetest__integration.t2.pk, "abcdef") + │ └─IndexRangeScan 12.50 cop[tikv] table:t2, index:PRIMARY(pk) range: decided by [eq(planner__core__casetest__integration.t2.pk, planner__core__casetest__integration.t1.col1)], keep order:false, stats:pseudo + └─Selection(Probe) 12.50 cop[tikv] in(planner__core__casetest__integration.t2.col1, "a", "b") + └─TableRowIDScan 12.50 cop[tikv] table:t2 keep order:false, stats:pseudo +explain format = 'brief' SELECT t1.pk FROM t1 LEFT JOIN t2 ON t1.col1 = t2.pk LEFT JOIN t3 ON t1.col3 = t3.pk WHERE t2.col1 IN ('a' , 'b') AND t3.keycol = 'c' AND t1.col2 = 'a' AND t1.col1 != 'abcdef' AND t1.col1 != 'aaaaaa'; +id estRows task access object operator info +IndexHashJoin 13.81 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t1.col1, inner key:planner__core__casetest__integration.t2.pk, equal cond:eq(planner__core__casetest__integration.t1.col1, planner__core__casetest__integration.t2.pk) +├─IndexHashJoin(Build) 12.50 root inner join, inner:IndexLookUp, outer key:planner__core__casetest__integration.t3.pk, inner key:planner__core__casetest__integration.t1.col3, equal cond:eq(planner__core__casetest__integration.t3.pk, planner__core__casetest__integration.t1.col3) +│ ├─IndexLookUp(Build) 10.00 root +│ │ ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t3, index:keycol(keycol, pad1, pad2) range:["c","c"], keep order:false, stats:pseudo +│ │ └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t3 keep order:false, stats:pseudo +│ └─IndexLookUp(Probe) 12.50 root +│ ├─Selection(Build) 18.10 cop[tikv] not(isnull(planner__core__casetest__integration.t1.col3)) +│ │ └─IndexRangeScan 18.12 cop[tikv] table:t1, index:col2(col2, col3) range: decided by [eq(planner__core__casetest__integration.t1.col3, planner__core__casetest__integration.t3.pk) eq(planner__core__casetest__integration.t1.col2, a)], keep order:false, stats:pseudo +│ └─Selection(Probe) 12.50 cop[tikv] ne(planner__core__casetest__integration.t1.col1, "aaaaaa"), ne(planner__core__casetest__integration.t1.col1, "abcdef"), not(isnull(planner__core__casetest__integration.t1.col1)) +│ └─TableRowIDScan 18.10 cop[tikv] table:t1 keep order:false, stats:pseudo +└─IndexLookUp(Probe) 12.50 root + ├─Selection(Build) 12.50 cop[tikv] ne(planner__core__casetest__integration.t2.pk, "aaaaaa"), ne(planner__core__casetest__integration.t2.pk, "abcdef") + │ └─IndexRangeScan 12.50 cop[tikv] table:t2, index:PRIMARY(pk) range: decided by [eq(planner__core__casetest__integration.t2.pk, planner__core__casetest__integration.t1.col1)], keep order:false, stats:pseudo + └─Selection(Probe) 12.50 cop[tikv] in(planner__core__casetest__integration.t2.col1, "a", "b") + └─TableRowIDScan 12.50 cop[tikv] table:t2 keep order:false, stats:pseudo +drop table if exists t1, t2; +create table t1(a int, b int, c int, d int, index idx_a_b_c(a, b, c)); +create table t2(a int, b int, c int, d int, index idx_a_b_c_d(a, b, c, d)); +explain format = 'brief' select a, b, c from t1 where a > 3 and b = 4 order by a, c; +id estRows task access object operator info +IndexReader 3.33 root index:Selection +└─Selection 3.33 cop[tikv] eq(planner__core__casetest__integration.t1.b, 4) + └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx_a_b_c(a, b, c) range:(3,+inf], keep order:true, stats:pseudo +explain format = 'brief' select * from t2 where a = 1 and c = 2 order by b, d; +id estRows task access object operator info +IndexReader 0.01 root index:Selection +└─Selection 0.01 cop[tikv] eq(planner__core__casetest__integration.t2.c, 2) + └─IndexRangeScan 10.00 cop[tikv] table:t2, index:idx_a_b_c_d(a, b, c, d) range:[1,1], keep order:true, stats:pseudo +explain format = 'brief' select a, b, c from t1 where (a = 1 and b = 1 and c = 1) or (a = 1 and b = 1 and c = 2) order by c; +id estRows task access object operator info +IndexReader 0.03 root index:IndexRangeScan +└─IndexRangeScan 0.03 cop[tikv] table:t1, index:idx_a_b_c(a, b, c) range:[1 1 1,1 1 2], keep order:true, stats:pseudo +explain format = 'brief' select a, b, c from t1 where (a = 1 and b = 1 and c < 3) or (a = 1 and b = 1 and c > 6) order by c; +id estRows task access object operator info +IndexReader 0.67 root index:IndexRangeScan +└─IndexRangeScan 0.67 cop[tikv] table:t1, index:idx_a_b_c(a, b, c) range:[1 1 -inf,1 1 3), (1 1 6,1 1 +inf], keep order:true, stats:pseudo +explain format = 'brief' select * from t2 where ((a = 1 and b = 1 and d < 3) or (a = 1 and b = 1 and d > 6)) and c = 3 order by d; +id estRows task access object operator info +IndexReader 0.00 root index:Selection +└─Selection 0.00 cop[tikv] eq(planner__core__casetest__integration.t2.c, 3), or(and(eq(planner__core__casetest__integration.t2.a, 1), and(eq(planner__core__casetest__integration.t2.b, 1), lt(planner__core__casetest__integration.t2.d, 3))), and(eq(planner__core__casetest__integration.t2.a, 1), and(eq(planner__core__casetest__integration.t2.b, 1), gt(planner__core__casetest__integration.t2.d, 6)))) + └─IndexRangeScan 10.00 cop[tikv] table:t2, index:idx_a_b_c_d(a, b, c, d) range:[1,1], keep order:true, stats:pseudo +drop table if exists t; +create table t(a int not null, b int not null); +explain format = 'brief' select * from t where exists (select 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a); +id estRows task access object operator info +HashJoin 8000.00 root semi join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +├─HashJoin(Build) 12500.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +│ ├─TableReader(Build) 10000.00 root data:TableFullScan +│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +│ └─TableReader(Probe) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where exists (select 1 from t t1 join t t2 on t1.a = t2.a and t1.a = t.a); +id estRows task access object operator info +HashJoin 8000.00 root semi join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +├─HashJoin(Build) 12500.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +│ ├─TableReader(Build) 10000.00 root data:TableFullScan +│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +│ └─TableReader(Probe) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a); +id estRows task access object operator info +HashJoin 10000.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +├─HashAgg(Build) 8000.00 root group by:planner__core__casetest__integration.t.a, funcs:firstrow(planner__core__casetest__integration.t.a)->planner__core__casetest__integration.t.a +│ └─HashJoin 12500.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +│ ├─TableReader(Build) 10000.00 root data:TableFullScan +│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +│ └─TableReader(Probe) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 on t1.a = t2.a and t1.a = t.a); +id estRows task access object operator info +HashJoin 10000.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +├─HashAgg(Build) 8000.00 root group by:planner__core__casetest__integration.t.a, funcs:firstrow(planner__core__casetest__integration.t.a)->planner__core__casetest__integration.t.a +│ └─HashJoin 12500.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +│ ├─TableReader(Build) 10000.00 root data:TableFullScan +│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +│ └─TableReader(Probe) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select /*+ hash_join_build(t) */ * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a); +id estRows task access object operator info +HashJoin 10000.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +├─HashAgg(Build) 8000.00 root group by:planner__core__casetest__integration.t.a, funcs:firstrow(planner__core__casetest__integration.t.a)->planner__core__casetest__integration.t.a +│ └─HashJoin 12500.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +│ ├─TableReader(Build) 10000.00 root data:TableFullScan +│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +│ └─TableReader(Probe) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select /*+ hash_join_probe(t) */ * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a); +id estRows task access object operator info +HashJoin 10000.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +├─HashAgg(Build) 8000.00 root group by:planner__core__casetest__integration.t.a, funcs:firstrow(planner__core__casetest__integration.t.a)->planner__core__casetest__integration.t.a +│ └─HashJoin 12500.00 root inner join, equal:[eq(planner__core__casetest__integration.t.a, planner__core__casetest__integration.t.a)] +│ ├─TableReader(Build) 10000.00 root data:TableFullScan +│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +│ └─TableReader(Probe) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +drop table if exists test; +create table test(id int, value int); +drop table if exists t; +create table t(c int); +insert t values(10), (8), (7), (9), (11); +explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id limit 1); +id estRows task access object operator info +HashAgg 1.00 root funcs:count(1)->Column#7 +└─HashJoin 7992.00 root semi join, equal:[eq(planner__core__casetest__integration.test.id, planner__core__casetest__integration.test.id)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.test.id)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.test.id)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id); +id estRows task access object operator info +HashAgg 1.00 root funcs:count(1)->Column#7 +└─HashJoin 7992.00 root semi join, equal:[eq(planner__core__casetest__integration.test.id, planner__core__casetest__integration.test.id)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.test.id)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.test.id)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select count(*) from test t1 where exists (select value from test t2 where t1.id = t2.id limit 1,2); +id estRows task access object operator info +HashAgg 1.00 root funcs:count(1)->Column#7 +└─Apply 10000.00 root CARTESIAN semi join + ├─TableReader(Build) 10000.00 root data:TableFullScan + │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo + └─Limit(Probe) 20000.00 root offset:1, count:2 + └─TableReader 30000.00 root data:Limit + └─Limit 30000.00 cop[tikv] offset:0, count:3 + └─Selection 30000.00 cop[tikv] eq(planner__core__casetest__integration.test.id, planner__core__casetest__integration.test.id) + └─TableFullScan 30000000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +explain format = 'brief' select * from t where 9 in (select c from t s where s.c < t.c limit 3); +id estRows task access object operator info +Apply 10000.00 root CARTESIAN semi join +├─TableReader(Build) 10000.00 root data:TableFullScan +│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +└─Selection(Probe) 24000.00 root eq(9, planner__core__casetest__integration.t.c) + └─Limit 30000.00 root offset:0, count:3 + └─TableReader 30000.00 root data:Limit + └─Limit 30000.00 cop[tikv] offset:0, count:3 + └─Selection 30000.00 cop[tikv] lt(planner__core__casetest__integration.t.c, planner__core__casetest__integration.t.c) + └─TableFullScan 37500.00 cop[tikv] table:s keep order:false, stats:pseudo +drop table if exists t0; +create table t0 (a int, b int, index(a, b)); +insert into t0 values (1, 1); +insert into t0 values (2, 2); +insert into t0 values (2, 2); +insert into t0 values (2, 2); +insert into t0 values (2, 2); +insert into t0 values (2, 2); +insert into t0 values (3, 3); +drop table if exists t1; +create table t1 (a int, b int, c int, index(a, b, c)); +drop table if exists t2; +create table t2 (a float, b float, index(a, b)); +drop table if exists t3; +create table t3 (a char(10), b char(10), c char(10), index(a, b, c)); +explain format = 'brief' select * from t0 where a > 1 and a < 3 order by b limit 2; +id estRows task access object operator info +Limit 2.00 root offset:0, count:2 +└─IndexReader 2.00 root index:Limit + └─Limit 2.00 cop[tikv] offset:0, count:2 + └─IndexRangeScan 2.50 cop[tikv] table:t0, index:a(a, b) range:[2,2], keep order:true, stats:pseudo +explain format = 'brief' select * from t1 where a >= 2 and a <= 2 and b = 2 and c > 2; +id estRows task access object operator info +IndexReader 0.33 root index:IndexRangeScan +└─IndexRangeScan 0.33 cop[tikv] table:t1, index:a(a, b, c) range:(2 2 2,2 2 +inf], keep order:false, stats:pseudo +explain format = 'brief' select * from t2 where a >= 2.5 and a <= 2.5 order by b limit 2; +id estRows task access object operator info +Limit 2.00 root offset:0, count:2 +└─IndexReader 2.00 root index:Limit + └─Limit 2.00 cop[tikv] offset:0, count:2 + └─IndexRangeScan 2.00 cop[tikv] table:t2, index:a(a, b) range:[2.5,2.5], keep order:true, stats:pseudo +explain format = 'brief' select * from t3 where a >= 'a' and a <= 'a' and b = 'b' and c > 'c'; +id estRows task access object operator info +IndexReader 0.33 root index:IndexRangeScan +└─IndexRangeScan 0.33 cop[tikv] table:t3, index:a(a, b, c) range:("a" "b" "c","a" "b" +inf], keep order:false, stats:pseudo +explain format = 'brief' select * from t1 where a is null or a in (1, 2, 3); +id estRows task access object operator info +IndexReader 260.00 root index:IndexRangeScan +└─IndexRangeScan 260.00 cop[tikv] table:t1, index:a(a, b, c) range:[NULL,NULL], [1,3], keep order:false, stats:pseudo +explain format = 'brief' select * from t1 where a is null and a = 1; +id estRows task access object operator info +TableDual 0.00 root rows:0 +drop table if exists t1; +CREATE TABLE t1 ( +key1 int(11) NOT NULL, +key2 int(11) NOT NULL, +key3 int(11) NOT NULL, +key4 int(11) NOT NULL, +key5 int(11) DEFAULT NULL, +key6 int(11) DEFAULT NULL, +key7 int(11) NOT NULL, +key8 int(11) NOT NULL, +KEY i1 (key1), +KEY i2 (key2), +KEY i3 (key3), +KEY i4 (key4), +KEY i5 (key5), +KEY i6 (key6) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +explain format = 'brief' SELECT /*+ use_index_merge(t1)*/ COUNT(*) FROM t1 WHERE (key4=42 AND key6 IS NOT NULL) OR (key1=4 AND key3=6); +id estRows task access object operator info +StreamAgg 1.00 root funcs:count(1)->Column#10 +└─IndexMerge 0.02 root type: union + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:i4(key4) range:[42,42], keep order:false, stats:pseudo + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:i1(key1) range:[4,4], keep order:false, stats:pseudo + └─Selection(Probe) 0.02 cop[tikv] or(and(eq(planner__core__casetest__integration.t1.key4, 42), not(isnull(planner__core__casetest__integration.t1.key6))), and(eq(planner__core__casetest__integration.t1.key1, 4), eq(planner__core__casetest__integration.t1.key3, 6))) + └─TableRowIDScan 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo +drop table if exists tpk; +create table tuk (a int, b int, c int, unique key (a, b, c)); +create table tik (a int, b int, c int, key (a, b, c)); +insert into tuk values (NULL, NULL, NULL); +insert into tik values (NULL, NULL, NULL); +insert into tuk values (NULL, NULL, NULL); +insert into tik values (NULL, NULL, NULL); +insert into tuk values (NULL, NULL, 1); +insert into tik values (NULL, NULL, 1); +insert into tuk values (NULL, NULL, 1); +insert into tik values (NULL, NULL, 1); +insert into tuk values (NULL, 1, NULL); +insert into tik values (NULL, 1, NULL); +insert into tuk values (NULL, 1, NULL); +insert into tik values (NULL, 1, NULL); +insert into tuk values (NULL, 1, 1); +insert into tik values (NULL, 1, 1); +insert into tuk values (NULL, 1, 1); +insert into tik values (NULL, 1, 1); +insert into tuk values (1, NULL, NULL); +insert into tik values (1, NULL, NULL); +insert into tuk values (1, NULL, NULL); +insert into tik values (1, NULL, NULL); +insert into tuk values (1, NULL, 1); +insert into tik values (1, NULL, 1); +insert into tuk values (1, NULL, 1); +insert into tik values (1, NULL, 1); +insert into tuk values (1, 1, NULL); +insert into tik values (1, 1, NULL); +insert into tuk values (1, 1, NULL); +insert into tik values (1, 1, NULL); +insert into tuk values (1, 1, 1); +insert into tik values (1, 1, 1); +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b=1; +id estRows task access object operator info +IndexReader_6 0.10 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo +select * from tuk where a<=>null and b=1; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b=1; +id estRows task access object operator info +IndexReader_7 0.01 root index:Selection_6 +└─Selection_6 0.01 cop[tikv] eq(planner__core__casetest__integration.tuk.b, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tuk where a<=>null and b=1; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b=1; +id estRows task access object operator info +IndexReader_6 0.10 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo +select * from tik where a<=>null and b=1; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b=1; +id estRows task access object operator info +IndexReader_7 0.01 root index:Selection_6 +└─Selection_6 0.01 cop[tikv] eq(planner__core__casetest__integration.tik.b, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tik where a<=>null and b=1; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b>0 and b<2; +id estRows task access object operator info +IndexReader_6 0.10 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo +select * from tuk where a<=>null and b>0 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b>0 and b<2; +id estRows task access object operator info +IndexReader_7 0.25 root index:Selection_6 +└─Selection_6 0.25 cop[tikv] eq(planner__core__casetest__integration.tuk.b, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tuk where a<=>null and b>0 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b>0 and b<2; +id estRows task access object operator info +IndexReader_6 0.10 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo +select * from tik where a<=>null and b>0 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b>0 and b<2; +id estRows task access object operator info +IndexReader_7 0.25 root index:Selection_6 +└─Selection_6 0.25 cop[tikv] eq(planner__core__casetest__integration.tik.b, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tik where a<=>null and b>0 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b>=1 and b<2; +id estRows task access object operator info +IndexReader_6 0.10 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo +select * from tuk where a<=>null and b>=1 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b>=1 and b<2; +id estRows task access object operator info +IndexReader_7 0.25 root index:Selection_6 +└─Selection_6 0.25 cop[tikv] eq(planner__core__casetest__integration.tuk.b, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tuk where a<=>null and b>=1 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b>=1 and b<2; +id estRows task access object operator info +IndexReader_6 0.10 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo +select * from tik where a<=>null and b>=1 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b>=1 and b<2; +id estRows task access object operator info +IndexReader_7 0.25 root index:Selection_6 +└─Selection_6 0.25 cop[tikv] eq(planner__core__casetest__integration.tik.b, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tik where a<=>null and b>=1 and b<2; +a b c +NULL 1 NULL +NULL 1 NULL +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b=1 and c=1; +id estRows task access object operator info +IndexReader_6 1.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1 1,NULL 1 1], keep order:false, stats:pseudo +select * from tuk where a<=>null and b=1 and c=1; +a b c +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b=1 and c=1; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] eq(planner__core__casetest__integration.tuk.b, 1), eq(planner__core__casetest__integration.tuk.c, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tuk where a<=>null and b=1 and c=1; +a b c +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b=1 and c=1; +id estRows task access object operator info +IndexReader_6 0.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1 1,NULL 1 1], keep order:false, stats:pseudo +select * from tik where a<=>null and b=1 and c=1; +a b c +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b=1 and c=1; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] eq(planner__core__casetest__integration.tik.b, 1), eq(planner__core__casetest__integration.tik.c, 1) + └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tik where a<=>null and b=1 and c=1; +a b c +NULL 1 1 +NULL 1 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a=1 and b<=>null and c=1; +id estRows task access object operator info +IndexReader_6 1.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[1 NULL 1,1 NULL 1], keep order:false, stats:pseudo +select * from tuk where a=1 and b<=>null and c=1; +a b c +1 NULL 1 +1 NULL 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a=1 and b<=>null and c=1; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] eq(planner__core__casetest__integration.tuk.c, 1) + └─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[1 NULL,1 NULL], keep order:false, stats:pseudo +select * from tuk where a=1 and b<=>null and c=1; +a b c +1 NULL 1 +1 NULL 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a=1 and b<=>null and c=1; +id estRows task access object operator info +IndexReader_6 0.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[1 NULL 1,1 NULL 1], keep order:false, stats:pseudo +select * from tik where a=1 and b<=>null and c=1; +a b c +1 NULL 1 +1 NULL 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a=1 and b<=>null and c=1; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] eq(planner__core__casetest__integration.tik.c, 1) + └─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[1 NULL,1 NULL], keep order:false, stats:pseudo +select * from tik where a=1 and b<=>null and c=1; +a b c +1 NULL 1 +1 NULL 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b<=>null and c=1; +id estRows task access object operator info +IndexReader_6 1.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL NULL 1,NULL NULL 1], keep order:false, stats:pseudo +select * from tuk where a<=>null and b<=>null and c=1; +a b c +NULL NULL 1 +NULL NULL 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b<=>null and c=1; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] eq(planner__core__casetest__integration.tuk.c, 1), nulleq(planner__core__casetest__integration.tuk.b, NULL) + └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tuk where a<=>null and b<=>null and c=1; +a b c +NULL NULL 1 +NULL NULL 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b<=>null and c=1; +id estRows task access object operator info +IndexReader_6 0.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL NULL 1,NULL NULL 1], keep order:false, stats:pseudo +select * from tik where a<=>null and b<=>null and c=1; +a b c +NULL NULL 1 +NULL NULL 1 +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b<=>null and c=1; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] eq(planner__core__casetest__integration.tik.c, 1), nulleq(planner__core__casetest__integration.tik.b, NULL) + └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tik where a<=>null and b<=>null and c=1; +a b c +NULL NULL 1 +NULL NULL 1 +set @@session.tidb_regard_null_as_point=true; +explain select * from tuk where a<=>null and b<=>null and c<=>null; +id estRows task access object operator info +IndexReader_6 1.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL NULL NULL,NULL NULL NULL], keep order:false, stats:pseudo +select * from tuk where a<=>null and b<=>null and c<=>null; +a b c +NULL NULL NULL +NULL NULL NULL +set @@session.tidb_regard_null_as_point=false; +explain select * from tuk where a<=>null and b<=>null and c<=>null; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] nulleq(planner__core__casetest__integration.tuk.b, NULL), nulleq(planner__core__casetest__integration.tuk.c, NULL) + └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tuk where a<=>null and b<=>null and c<=>null; +a b c +NULL NULL NULL +NULL NULL NULL +set @@session.tidb_regard_null_as_point=true; +explain select * from tik where a<=>null and b<=>null and c<=>null; +id estRows task access object operator info +IndexReader_6 0.00 root index:IndexRangeScan_5 +└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL NULL NULL,NULL NULL NULL], keep order:false, stats:pseudo +select * from tik where a<=>null and b<=>null and c<=>null; +a b c +NULL NULL NULL +NULL NULL NULL +set @@session.tidb_regard_null_as_point=false; +explain select * from tik where a<=>null and b<=>null and c<=>null; +id estRows task access object operator info +IndexReader_7 0.00 root index:Selection_6 +└─Selection_6 0.00 cop[tikv] nulleq(planner__core__casetest__integration.tik.b, NULL), nulleq(planner__core__casetest__integration.tik.c, NULL) + └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo +select * from tik where a<=>null and b<=>null and c<=>null; +a b c +NULL NULL NULL +NULL NULL NULL +set @@session.tidb_regard_null_as_point=default; +drop table if exists t1; +create table t1(c1 varchar(100), c2 varchar(100), key(c1), key(c2), c3 varchar(100)); +insert into t1 values('ab', '10', '10'); +drop table if exists tt1; +create table tt1(c1 varchar(100), c2 varchar(100), c3 varchar(100), c4 varchar(100), key idx_0(c1), key idx_1(c2, c3)); +insert into tt1 values('ab', '10', '10', '10'); +drop table if exists tt2; +create table tt2 (c1 int , pk int, primary key( pk ) , unique key( c1)); +insert into tt2 values(-3896405, -1), (-2, 1), (-1, -2); +drop table if exists tt3; +create table tt3(c1 int, c2 int, c3 int as (c1 + c2), key(c1), key(c2), key(c3)); +insert into tt3(c1, c2) values(1, 1); +select @@tidb_enable_index_merge; +@@tidb_enable_index_merge +1 +set tidb_enable_index_merge = on; +explain format=brief select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab'; +id estRows task access object operator info +Projection 15.99 root 1->Column#5 +└─Selection 15.99 root or(eq(planner__core__casetest__integration.t1.c1, "de"), and(eq(planner__core__casetest__integration.t1.c2, "10"), eq(from_base64(to_base64(planner__core__casetest__integration.t1.c1)), "ab"))) + └─IndexMerge 19.99 root type: union + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c1(c1) range:["de","de"], keep order:false, stats:pseudo + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c2(c2) range:["10","10"], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo +select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab'; +1 +1 +explain format=brief select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'ab' or c2 = '10' and char_length(left(c1, 10)) = 10; +id estRows task access object operator info +Projection 17.99 root 1->Column#5 +└─Selection 0.04 root or(eq(planner__core__casetest__integration.t1.c1, "ab"), and(eq(planner__core__casetest__integration.t1.c2, "10"), eq(char_length(left(planner__core__casetest__integration.t1.c1, 10)), 10))) + └─IndexMerge 19.99 root type: union + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c1(c1) range:["ab","ab"], keep order:false, stats:pseudo + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c2(c2) range:["10","10"], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo +select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'ab' or c2 = '10' and char_length(left(c1, 10)) = 10; +1 +1 +explain format=brief select /*+ use_index_merge(tt1) */ 1 from tt1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c3)) = '10'; +id estRows task access object operator info +Projection 15.99 root 1->Column#6 +└─Selection 15.99 root or(eq(planner__core__casetest__integration.tt1.c1, "de"), and(eq(planner__core__casetest__integration.tt1.c2, "10"), eq(from_base64(to_base64(planner__core__casetest__integration.tt1.c3)), "10"))) + └─IndexMerge 19.99 root type: union + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:tt1, index:idx_0(c1) range:["de","de"], keep order:false, stats:pseudo + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:tt1, index:idx_1(c2, c3) range:["10","10"], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 19.99 cop[tikv] table:tt1 keep order:false, stats:pseudo +select /*+ use_index_merge(tt1) */ 1 from tt1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c3)) = '10'; +1 +1 +explain format=brief select /*+ use_index_merge( tt2 ) */ 1 from tt2 where tt2.c1 in (-3896405) or tt2.pk in (1, 53330) and to_base64(left(pk, 5)); +id estRows task access object operator info +Projection 2.40 root 1->Column#3 +└─Selection 2.40 root or(eq(planner__core__casetest__integration.tt2.c1, -3896405), and(in(planner__core__casetest__integration.tt2.pk, 1, 53330), istrue_with_null(cast(to_base64(left(cast(planner__core__casetest__integration.tt2.pk, var_string(20)), 5)), double BINARY)))) + └─IndexMerge 3.00 root type: union + ├─IndexRangeScan(Build) 1.00 cop[tikv] table:tt2, index:c1(c1) range:[-3896405,-3896405], keep order:false, stats:pseudo + ├─TableRangeScan(Build) 2.00 cop[tikv] table:tt2 range:[1,1], [53330,53330], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 3.00 cop[tikv] table:tt2 keep order:false, stats:pseudo +select /*+ use_index_merge( tt2 ) */ 1 from tt2 where tt2.c1 in (-3896405) or tt2.pk in (1, 53330) and to_base64(left(pk, 5)); +1 +1 +explain format=brief select /*+ use_index_merge(tt3) */ 1 from tt3 where c1 < -10 or c2 < 10 and reverse(c3) = '2'; +id estRows task access object operator info +Projection 5098.44 root 1->Column#5 +└─Selection 2825.66 root or(lt(planner__core__casetest__integration.tt3.c1, -10), and(lt(planner__core__casetest__integration.tt3.c2, 10), eq(reverse(cast(planner__core__casetest__integration.tt3.c3, var_string(20))), "2"))) + └─IndexMerge 5542.21 root type: union + ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:tt3, index:c1(c1) range:[-inf,-10), keep order:false, stats:pseudo + ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:tt3, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 5542.21 cop[tikv] table:tt3 keep order:false, stats:pseudo +select /*+ use_index_merge(tt3) */ 1 from tt3 where c1 < -10 or c2 < 10 and reverse(c3) = '2'; +1 +1 +explain format=brief select 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab'; +id estRows task access object operator info +Projection 8000.00 root 1->Column#5 +└─Selection 8000.00 root or(eq(planner__core__casetest__integration.t1.c1, "de"), and(eq(planner__core__casetest__integration.t1.c2, "10"), eq(from_base64(to_base64(planner__core__casetest__integration.t1.c1)), "ab"))) + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +select 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab'; +1 +1 +set tidb_enable_index_merge = 1; +set tidb_enable_index_merge = default; +drop table if exists t1,t2; +create table t1(a int); +create table t2(a int, b int, c int, primary key(a,b) nonclustered); +explain format = 'brief' select (select c from t2 where t2.a = t1.a and t2.b = 1) from t1; +id estRows task access object operator info +HashJoin 10000.00 root left outer join, equal:[eq(planner__core__casetest__integration.t1.a, planner__core__casetest__integration.t2.a)] +├─TableReader(Build) 10.00 root data:Selection +│ └─Selection 10.00 cop[tikv] eq(planner__core__casetest__integration.t2.b, 1) +│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +└─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select (select c from t2 where t2.a = t1.a and (t2.b = 1 or t2.b = 2)) from t1; +id estRows task access object operator info +Projection 10000.00 root planner__core__casetest__integration.t2.c +└─Apply 10000.00 root CARTESIAN left outer join + ├─TableReader(Build) 10000.00 root data:TableFullScan + │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo + └─MaxOneRow(Probe) 10000.00 root + └─IndexLookUp 200.00 root + ├─Selection(Build) 200.00 cop[tikv] or(eq(planner__core__casetest__integration.t2.b, 1), eq(planner__core__casetest__integration.t2.b, 2)) + │ └─IndexRangeScan 100000.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(planner__core__casetest__integration.t2.a, planner__core__casetest__integration.t1.a)], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 200.00 cop[tikv] table:t2 keep order:false, stats:pseudo +set tidb_cost_model_version=2; +drop sequence if exists s1, s2; +create sequence s1; +create sequence s2; +explain format = 'brief' select 1 from s1; +id estRows task access object operator info +Projection 1.00 root 1->Column#1 +└─TableDual 1.00 root rows:1 +explain format = 'brief' select count(1) from s1; +id estRows task access object operator info +StreamAgg 1.00 root funcs:count(1)->Column#1 +└─TableDual 1.00 root rows:1 +explain format = 'brief' select count(*) from s1; +id estRows task access object operator info +StreamAgg 1.00 root funcs:count(1)->Column#1 +└─TableDual 1.00 root rows:1 +explain format = 'brief' select sum(1) from s1; +id estRows task access object operator info +StreamAgg 1.00 root funcs:sum(1)->Column#1 +└─TableDual 1.00 root rows:1 +explain format = 'brief' select count(1) as cnt from s1 union select count(1) as cnt from s2; +id estRows task access object operator info +HashAgg 2.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 2.00 root + ├─StreamAgg 1.00 root funcs:count(1)->Column#1 + │ └─TableDual 1.00 root rows:1 + └─StreamAgg 1.00 root funcs:count(1)->Column#2 + └─TableDual 1.00 root rows:1 +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int, b int, c int, d int, e int, f int, g int, primary key (a), unique key c_d_e (c, d, e), unique key f (f), unique key f_g (f, g), key g (g)); +set @@tidb_enable_chunk_rpc = on; +explain format = 'verbose' select * from t where a > 1 order by f; +id estRows estCost task access object operator info +Sort_5 3333.33 2146348.14 root planner__core__casetest__integration.t.f +└─TableReader_9 3333.33 160128.74 root data:TableRangeScan_8 + └─TableRangeScan_8 3333.33 923531.15 cop[tikv] table:t range:(1,+inf], keep order:false, stats:pseudo +Level Code Message +Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [{planner__core__casetest__integration.t.f asc}], TaskTp: rootTask} +Note 1105 [t] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask} +explain format = 'verbose' select * from t where f > 1; +id estRows estCost task access object operator info +TableReader_7 3333.33 316532.90 root data:Selection_6 +└─Selection_6 3333.33 3269593.45 cop[tikv] gt(planner__core__casetest__integration.t.f, 1) + └─TableFullScan_5 10000.00 2770593.45 cop[tikv] table:t keep order:false, stats:pseudo +Level Code Message +Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask} +explain format = 'verbose' select f from t where f > 1; +id estRows estCost task access object operator info +IndexReader_6 3333.33 50257.78 root index:IndexRangeScan_5 +└─IndexRangeScan_5 3333.33 542666.67 cop[tikv] table:t, index:f(f) range:(1,+inf], keep order:false, stats:pseudo +Level Code Message +Note 1105 [f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask} +explain format = 'verbose' select * from t where f > 3 and g = 5; +id estRows estCost task access object operator info +IndexLookUp_15 3.33 19551.99 root +├─IndexRangeScan_12(Build) 10.00 2035.00 cop[tikv] table:t, index:g(g) range:[5,5], keep order:false, stats:pseudo +└─Selection_14(Probe) 3.33 3269.59 cop[tikv] gt(planner__core__casetest__integration.t.f, 3) + └─TableRowIDScan_13 10.00 2770.59 cop[tikv] table:t keep order:false, stats:pseudo +Level Code Message +Note 1105 [t,f_g,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask} +explain format = 'verbose' select * from t where g = 5 order by f; +id estRows estCost task access object operator info +Sort_5 10.00 21321.97 root planner__core__casetest__integration.t.f +└─IndexLookUp_13 10.00 19545.34 root + ├─IndexRangeScan_11(Build) 10.00 2035.00 cop[tikv] table:t, index:g(g) range:[5,5], keep order:false, stats:pseudo + └─TableRowIDScan_12(Probe) 10.00 2770.59 cop[tikv] table:t keep order:false, stats:pseudo +Level Code Message +Note 1105 [t,f_g,g] remain after pruning paths for t given Prop{SortItems: [{planner__core__casetest__integration.t.f asc}], TaskTp: rootTask} +Note 1105 [t,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask} +explain format = 'verbose' select * from t where d = 3 order by c, e; +id estRows estCost task access object operator info +IndexLookUp_15 10.00 215519.24 root +├─Selection_14(Build) 10.00 2941000.00 cop[tikv] eq(planner__core__casetest__integration.t.d, 3) +│ └─IndexFullScan_12 10000.00 2442000.00 cop[tikv] table:t, index:c_d_e(c, d, e) keep order:true, stats:pseudo +└─TableRowIDScan_13(Probe) 10.00 2770.59 cop[tikv] table:t keep order:false, stats:pseudo +Level Code Message +Note 1105 [t,c_d_e] remain after pruning paths for t given Prop{SortItems: [{planner__core__casetest__integration.t.c asc} {planner__core__casetest__integration.t.e asc}], TaskTp: rootTask} +Note 1105 [t] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask} +set @@tidb_enable_chunk_rpc = default; +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int unsigned primary key, b int, c int, index idx_b(b)); +insert into t values (1,2,3), (4,5,6), (7,8,9), (10,11,12), (13,14,15); +analyze table t; +set @@tidb_enable_chunk_rpc = on; +set tidb_opt_prefer_range_scan = 0; +explain format = 'verbose' select * from t where b > 5; +id estRows estCost task access object operator info +TableReader_7 3.00 130.42 root data:Selection_6 +└─Selection_6 3.00 1386.04 cop[tikv] gt(planner__core__casetest__integration.t.b, 5) + └─TableFullScan_5 5.00 1136.54 cop[tikv] table:t keep order:false +explain format = 'verbose' select * from t where b = 6 order by a limit 1; +id estRows estCost task access object operator info +Limit_11 1.00 98.74 root offset:0, count:1 +└─TableReader_24 1.00 98.74 root data:Limit_23 + └─Limit_23 1.00 1386.04 cop[tikv] offset:0, count:1 + └─Selection_22 1.00 1386.04 cop[tikv] eq(planner__core__casetest__integration.t.b, 6) + └─TableFullScan_21 5.00 1136.54 cop[tikv] table:t keep order:true +explain format = 'verbose' select * from t where b = 6 limit 1; +id estRows estCost task access object operator info +Limit_8 1.00 98.74 root offset:0, count:1 +└─TableReader_13 1.00 98.74 root data:Limit_12 + └─Limit_12 1.00 1386.04 cop[tikv] offset:0, count:1 + └─Selection_11 1.00 1386.04 cop[tikv] eq(planner__core__casetest__integration.t.b, 6) + └─TableFullScan_10 5.00 1136.54 cop[tikv] table:t keep order:false +set tidb_opt_prefer_range_scan = 1; +explain format = 'verbose' select * from t where b > 5; +id estRows estCost task access object operator info +IndexLookUp_7 3.00 5856.46 root +├─IndexRangeScan_5(Build) 3.00 610.50 cop[tikv] table:t, index:idx_b(b) range:(5,+inf], keep order:false +└─TableRowIDScan_6(Probe) 3.00 681.92 cop[tikv] table:t keep order:false +Level Code Message +Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask} +explain format = 'verbose' select * from t where b = 6 order by a limit 1; +id estRows estCost task access object operator info +TopN_9 1.00 1956.63 root planner__core__casetest__integration.t.a, offset:0, count:1 +└─IndexLookUp_16 1.00 1951.83 root + ├─TopN_15(Build) 1.00 206.70 cop[tikv] planner__core__casetest__integration.t.a, offset:0, count:1 + │ └─IndexRangeScan_13 1.00 203.50 cop[tikv] table:t, index:idx_b(b) range:[6,6], keep order:false + └─TableRowIDScan_14(Probe) 1.00 186.61 cop[tikv] table:t keep order:false +Level Code Message +Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: copMultiReadTask} +explain format = 'verbose' select * from t where b = 6 limit 1; +id estRows estCost task access object operator info +IndexLookUp_13 1.00 1170.97 root limit embedded(offset:0, count:1) +├─Limit_12(Build) 1.00 203.50 cop[tikv] offset:0, count:1 +│ └─IndexRangeScan_10 1.00 203.50 cop[tikv] table:t, index:idx_b(b) range:[6,6], keep order:false +└─TableRowIDScan_11(Probe) 1.00 186.61 cop[tikv] table:t keep order:false +Level Code Message +Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: copMultiReadTask} +set @@tidb_enable_chunk_rpc = default; +set tidb_opt_prefer_range_scan = default; +drop table if exists t; +create table t(a int primary key, b int, c int, index idx_b(b)); +insert into t values (1,2,3), (4,5,6), (7,8,9), (10, 11, 12), (13,14,15), (16, 17, 18); +analyze table t; +explain format = 'brief' select * from t use index (idx_b) where b = 2 limit 1; +id estRows task access object operator info +IndexLookUp 1.00 root limit embedded(offset:0, count:1) +├─Limit(Build) 1.00 cop[tikv] offset:0, count:1 +│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:idx_b(b) range:[2,2], keep order:false +└─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false +drop table if exists t1; +create table t1(c1 int); +insert into t1 values(1), (2), (3), (4), (5), (6); +select floor(dt.rn/2) rownum, count(c1) from (select @rownum := @rownum + 1 rn, c1 from (select @rownum := -1) drn, t1) dt group by floor(dt.rn/2) order by rownum; +rownum count(c1) +0 2 +1 2 +2 2 +create table ta(a int, b int); +set sql_mode=''; +explain format = 'brief' select floor(dt.rn/2) rownum, count(c1) from (select @rownum := @rownum + 1 rn, c1 from (select @rownum := -1) drn, t1) dt group by floor(dt.rn/2) order by rownum; +id estRows task access object operator info +Sort 1.00 root Column#6 +└─Projection 1.00 root floor(div(cast(Column#4, decimal(20,0) BINARY), 2))->Column#6, Column#5 + └─HashAgg 1.00 root group by:Column#13, funcs:count(Column#11)->Column#5, funcs:firstrow(Column#12)->Column#4 + └─Projection 10000.00 root planner__core__casetest__integration.t1.c1->Column#11, Column#4->Column#12, floor(div(cast(Column#4, decimal(20,0) BINARY), 2))->Column#13 + └─Projection 10000.00 root setvar(rownum, plus(getvar(rownum), 1))->Column#4, planner__core__casetest__integration.t1.c1 + └─HashJoin 10000.00 root CARTESIAN inner join + ├─Projection(Build) 1.00 root setvar(rownum, -1)->Column#1 + │ └─TableDual 1.00 root rows:1 + └─TableReader(Probe) 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select @n:=@n+1 as e from ta group by e; +id estRows task access object operator info +Projection 1.00 root setvar(n, plus(getvar(n), 1))->Column#4 +└─HashAgg 1.00 root group by:Column#8, funcs:firstrow(1)->Column#7 + └─Projection 10000.00 root setvar(n, plus(cast(getvar(n), double BINARY), 1))->Column#8 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo +explain format = 'brief' select @n:=@n+a as e from ta group by e; +id estRows task access object operator info +Projection 8000.00 root setvar(n, plus(getvar(n), cast(planner__core__casetest__integration.ta.a, double BINARY)))->Column#4 +└─HashAgg 8000.00 root group by:Column#7, funcs:firstrow(Column#6)->planner__core__casetest__integration.ta.a + └─Projection 10000.00 root planner__core__casetest__integration.ta.a->Column#6, setvar(n, plus(getvar(n), cast(planner__core__casetest__integration.ta.a, double BINARY)))->Column#7 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo +explain format = 'brief' select * from (select @n:=@n+1 as e from ta) tt group by e; +id estRows task access object operator info +HashAgg 1.00 root group by:Column#4, funcs:firstrow(Column#4)->Column#4 +└─Projection 10000.00 root setvar(n, plus(getvar(n), 1))->Column#4 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo +explain format = 'brief' select * from (select @n:=@n+a as e from ta) tt group by e; +id estRows task access object operator info +HashAgg 8000.00 root group by:Column#4, funcs:firstrow(Column#4)->Column#4 +└─Projection 10000.00 root setvar(n, plus(getvar(n), cast(planner__core__casetest__integration.ta.a, double BINARY)))->Column#4 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo +explain format = 'brief' select a from ta group by @n:=@n+1; +id estRows task access object operator info +HashAgg 1.00 root group by:Column#5, funcs:firstrow(Column#4)->planner__core__casetest__integration.ta.a +└─Projection 10000.00 root planner__core__casetest__integration.ta.a->Column#4, setvar(n, plus(getvar(n), 1))->Column#5 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo +explain format = 'brief' select a from ta group by @n:=@n+a; +id estRows task access object operator info +HashAgg 8000.00 root group by:Column#5, funcs:firstrow(Column#4)->planner__core__casetest__integration.ta.a +└─Projection 10000.00 root planner__core__casetest__integration.ta.a->Column#4, setvar(n, plus(getvar(n), cast(planner__core__casetest__integration.ta.a, double BINARY)))->Column#5 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo +set sql_mode=default; +drop table if exists t1, t2, t3, t4, t5, t6, t7, t8; +create table t1 (a bigint key); +create table t2 (a int key); +create definer=`root`@`127.0.0.1` view v1 as (select a from t1) union (select a from t2); +create table t3 (a varchar(100) key); +create table t4 (a varchar(10) key); +create definer=`root`@`127.0.0.1` view v2 as (select a from t3) union (select a from t4); +create table t5 (a char(100) key); +create table t6 (a char(10) key); +create definer=`root`@`127.0.0.1` view v3 as (select a from t5) union (select a from t6); +create table t7 (a varchar(100) key); +create table t8 (a int key); +create definer=`root`@`127.0.0.1` view v4 as (select a from t7) union (select a from t8); +explain format='brief' select * from v1 where a = 1; -- the condition should be downcast through both side and go get point; +id estRows task access object operator info +HashAgg 2.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 2.00 root + ├─Point_Get 1.00 root table:t1 handle:1 + └─Projection 1.00 root cast(planner__core__casetest__integration.t2.a, bigint(20) BINARY)->Column#3 + └─Point_Get 1.00 root table:t2 handle:1 +select * from v1 where a = 1; -- the condition should be downcast through both side and go get point; +a +explain format='brief' select * from v1 where a = '1test'; -- the condition should be downcast through both side and go get point too; +id estRows task access object operator info +HashAgg 2.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 2.00 root + ├─Point_Get 1.00 root table:t1 handle:1 + └─Projection 1.00 root cast(planner__core__casetest__integration.t2.a, bigint(20) BINARY)->Column#3 + └─Point_Get 1.00 root table:t2 handle:1 +select * from v1 where a = '1test'; -- the condition should be downcast through both side and go get point too; +a +explain format='brief' select * from v1 where a > 1; -- the condition should be downcast through both side and go range scan; +id estRows task access object operator info +HashAgg 5333.33 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 6666.67 root + ├─TableReader 3333.33 root data:TableRangeScan + │ └─TableRangeScan 3333.33 cop[tikv] table:t1 range:(1,+inf], keep order:false, stats:pseudo + └─Projection 3333.33 root cast(planner__core__casetest__integration.t2.a, bigint(20) BINARY)->Column#3 + └─TableReader 3333.33 root data:TableRangeScan + └─TableRangeScan 3333.33 cop[tikv] table:t2 range:(1,+inf], keep order:false, stats:pseudo +select * from v1 where a > 1; -- the condition should be downcast through both side and go range scan; +a +explain format='brief' select * from v2 where a = 'test'; +id estRows task access object operator info +HashAgg 16.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 20.00 root + ├─Point_Get 1.00 root table:t3, clustered index:PRIMARY(a) + └─Projection 10.00 root cast(planner__core__casetest__integration.t4.a, varchar(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3 + └─Point_Get 1.00 root table:t4, clustered index:PRIMARY(a) +select * from v2 where a = 'test'; +a +explain format='brief' select * from v2 where a = 1; +id estRows task access object operator info +HashAgg 12800.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 16000.00 root + ├─TableReader 8000.00 root data:Selection + │ └─Selection 8000.00 cop[tikv] eq(cast(planner__core__casetest__integration.t3.a, double BINARY), 1) + │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo + └─Projection 8000.00 root cast(planner__core__casetest__integration.t4.a, varchar(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3 + └─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] eq(cast(cast(planner__core__casetest__integration.t4.a, varchar(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), double BINARY), 1) + └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo +select * from v2 where a = 1; +a +explain format='brief' select * from v2 where a > 'test'; +id estRows task access object operator info +HashAgg 5333.33 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 6666.67 root + ├─TableReader 3333.33 root data:TableRangeScan + │ └─TableRangeScan 3333.33 cop[tikv] table:t3 range:("test",+inf], keep order:false, stats:pseudo + └─Projection 3333.33 root cast(planner__core__casetest__integration.t4.a, varchar(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3 + └─TableReader 3333.33 root data:TableRangeScan + └─TableRangeScan 3333.33 cop[tikv] table:t4 range:("test",+inf], keep order:false, stats:pseudo +select * from v2 where a > 'test'; +a +explain format='brief' select * from v3 where a = 'test' -- the condition shouldn't be downcast through both side and go get point; +id estRows task access object operator info +HashAgg 6408.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 8010.00 root + ├─Point_Get 1.00 root table:t5, clustered index:PRIMARY(a) + └─Projection 8000.00 root cast(planner__core__casetest__integration.t6.a, char(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3 + └─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] eq(cast(planner__core__casetest__integration.t6.a, char(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), "test") + └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo +select * from v3 where a = 'test' -- the condition shouldn't be downcast through both side and go get point; +a +explain format='brief' select * from v3 where a > 'test' -- the condition shouldn't be downcast through both side and go get point too; +id estRows task access object operator info +HashAgg 9066.67 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 11333.33 root + ├─TableReader 3333.33 root data:TableRangeScan + │ └─TableRangeScan 3333.33 cop[tikv] table:t5 range:("test",+inf], keep order:false, stats:pseudo + └─Projection 8000.00 root cast(planner__core__casetest__integration.t6.a, char(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3 + └─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] gt(cast(planner__core__casetest__integration.t6.a, char(100) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), "test") + └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo +select * from v3 where a > 'test' -- the condition shouldn't be downcast through both side and go get point too; +a +explain format='brief' select * from v4 where a = 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range; +id estRows task access object operator info +HashAgg 6408.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 8010.00 root + ├─Point_Get 1.00 root table:t7, clustered index:PRIMARY(a) + └─Selection 8000.00 root eq(Column#3, "test") + └─Projection 10000.00 root cast(planner__core__casetest__integration.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo +select * from v4 where a = 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range; +a +explain format='brief' select * from v4 where a > 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range; +id estRows task access object operator info +HashAgg 9066.67 root group by:Column#3, funcs:firstrow(Column#3)->Column#3 +└─Union 11333.33 root + ├─TableReader 3333.33 root data:TableRangeScan + │ └─TableRangeScan 3333.33 cop[tikv] table:t7 range:("test",+inf], keep order:false, stats:pseudo + └─Selection 8000.00 root gt(Column#3, "test") + └─Projection 10000.00 root cast(planner__core__casetest__integration.t8.a, varchar(100) BINARY CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#3 + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo +select * from v4 where a > 'test' -- diff column union may have precision loss couldn't downcast the condition to get the range; +a +set tidb_cost_model_version=2; +drop table if exists t; +create table t (id int, value decimal(10,5)); +desc format = 'brief' select count(*) from t join (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 1) v on v.id = t.id and v.v1 = t.value; +id estRows task access object operator info +StreamAgg 1.00 root funcs:count(1)->Column#10 +└─HashJoin 1.00 root inner join, equal:[eq(planner__core__casetest__integration.t.id, planner__core__casetest__integration.t.id) eq(planner__core__casetest__integration.t.value, planner__core__casetest__integration.t.value)] + ├─Selection(Build) 0.80 root not(isnull(planner__core__casetest__integration.t.id)), not(isnull(planner__core__casetest__integration.t.value)) + │ └─TopN 1.00 root planner__core__casetest__integration.t.value, offset:0, count:1 + │ └─HashJoin 12487.50 root inner join, equal:[eq(planner__core__casetest__integration.t.id, planner__core__casetest__integration.t.id)] + │ ├─TableReader(Build) 9990.00 root data:Selection + │ │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.id)) + │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo + │ └─TableReader(Probe) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.id)) + │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo + └─TableReader(Probe) 9980.01 root data:Selection + └─Selection 9980.01 cop[tikv] not(isnull(planner__core__casetest__integration.t.id)), not(isnull(planner__core__casetest__integration.t.value)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select count(*) from t join (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 1) v on v.id = t.id and v.v1 = t.value; +id estRows task access object operator info +StreamAgg 1.00 root funcs:count(1)->Column#10 +└─HashJoin 1.00 root inner join, equal:[eq(planner__core__casetest__integration.t.id, planner__core__casetest__integration.t.id) eq(planner__core__casetest__integration.t.value, planner__core__casetest__integration.t.value)] + ├─Selection(Build) 0.80 root not(isnull(planner__core__casetest__integration.t.id)), not(isnull(planner__core__casetest__integration.t.value)) + │ └─TopN 1.00 root planner__core__casetest__integration.t.value, offset:0, count:1 + │ └─HashJoin 12487.50 root inner join, equal:[eq(planner__core__casetest__integration.t.id, planner__core__casetest__integration.t.id)] + │ ├─TableReader(Build) 9990.00 root data:Selection + │ │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.id)) + │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo + │ └─TableReader(Probe) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(planner__core__casetest__integration.t.id)) + │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo + └─TableReader(Probe) 9980.01 root data:Selection + └─Selection 9980.01 cop[tikv] not(isnull(planner__core__casetest__integration.t.id)), not(isnull(planner__core__casetest__integration.t.value)) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +set tidb_partition_prune_mode='dynamic'; +drop table if exists t1, t2_part; +create table t1 (a int, b int); +create table t2_part (a int, b int, key(a)) partition by hash(a) partitions 4; +set @@tidb_skip_missing_partition_stats = off; +set @@tidb_opt_fix_control = ""; +explain select /*+ TIDB_INLJ(t2_part@sel_2) */ * from t1 where t1.b<10 and not exists (select 1 from t2_part where t1.a=t2_part.a and t2_part.b<20); +id estRows task access object operator info +HashJoin_19 2658.67 root anti semi join, equal:[eq(planner__core__casetest__integration.t1.a, planner__core__casetest__integration.t2_part.a)] +├─PartitionUnion_23(Build) 13293.33 root +│ ├─TableReader_31 3323.33 root data:Projection_25 +│ │ └─Projection_25 3323.33 cop[tikv] planner__core__casetest__integration.t2_part.a +│ │ └─Selection_30 3323.33 cop[tikv] lt(planner__core__casetest__integration.t2_part.b, 20) +│ │ └─TableFullScan_29 10000.00 cop[tikv] table:t2_part, partition:p0 keep order:false, stats:pseudo +│ ├─TableReader_39 3323.33 root data:Projection_33 +│ │ └─Projection_33 3323.33 cop[tikv] planner__core__casetest__integration.t2_part.a +│ │ └─Selection_38 3323.33 cop[tikv] lt(planner__core__casetest__integration.t2_part.b, 20) +│ │ └─TableFullScan_37 10000.00 cop[tikv] table:t2_part, partition:p1 keep order:false, stats:pseudo +│ ├─TableReader_47 3323.33 root data:Projection_41 +│ │ └─Projection_41 3323.33 cop[tikv] planner__core__casetest__integration.t2_part.a +│ │ └─Selection_46 3323.33 cop[tikv] lt(planner__core__casetest__integration.t2_part.b, 20) +│ │ └─TableFullScan_45 10000.00 cop[tikv] table:t2_part, partition:p2 keep order:false, stats:pseudo +│ └─TableReader_55 3323.33 root data:Projection_49 +│ └─Projection_49 3323.33 cop[tikv] planner__core__casetest__integration.t2_part.a +│ └─Selection_54 3323.33 cop[tikv] lt(planner__core__casetest__integration.t2_part.b, 20) +│ └─TableFullScan_53 10000.00 cop[tikv] table:t2_part, partition:p3 keep order:false, stats:pseudo +└─TableReader_22(Probe) 3323.33 root data:Selection_21 + └─Selection_21 3323.33 cop[tikv] lt(planner__core__casetest__integration.t1.b, 10) + └─TableFullScan_20 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +Level Code Message +Warning 1105 disable dynamic pruning due to t2_part has no global stats +Warning 1815 Optimizer Hint /*+ INL_JOIN(t2_part) */ or /*+ TIDB_INLJ(t2_part) */ is inapplicable +set @@tidb_opt_fix_control = "44262:ON"; +explain select /*+ TIDB_INLJ(t2_part@sel_2) */ * from t1 where t1.b<10 and not exists (select 1 from t2_part where t1.a=t2_part.a and t2_part.b<20); +id estRows task access object operator info +IndexJoin_13 2658.67 root anti semi join, inner:IndexLookUp_12, outer key:planner__core__casetest__integration.t1.a, inner key:planner__core__casetest__integration.t2_part.a, equal cond:eq(planner__core__casetest__integration.t1.a, planner__core__casetest__integration.t2_part.a) +├─TableReader_18(Build) 3323.33 root data:Selection_17 +│ └─Selection_17 3323.33 cop[tikv] lt(planner__core__casetest__integration.t1.b, 10) +│ └─TableFullScan_16 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─IndexLookUp_12(Probe) 4154.17 root partition:all + ├─IndexRangeScan_9(Build) 12500.00 cop[tikv] table:t2_part, index:a(a) range: decided by [eq(planner__core__casetest__integration.t2_part.a, planner__core__casetest__integration.t1.a)], keep order:false, stats:pseudo + └─Selection_11(Probe) 4154.17 cop[tikv] lt(planner__core__casetest__integration.t2_part.b, 20) + └─TableRowIDScan_10 12500.00 cop[tikv] table:t2_part keep order:false, stats:pseudo +set @@tidb_opt_fix_control = ""; +set @@tidb_skip_missing_partition_stats = on; +explain select /*+ TIDB_INLJ(t2_part@sel_2) */ * from t1 where t1.b<10 and not exists (select 1 from t2_part where t1.a=t2_part.a and t2_part.b<20); +id estRows task access object operator info +IndexJoin_13 2658.67 root anti semi join, inner:IndexLookUp_12, outer key:planner__core__casetest__integration.t1.a, inner key:planner__core__casetest__integration.t2_part.a, equal cond:eq(planner__core__casetest__integration.t1.a, planner__core__casetest__integration.t2_part.a) +├─TableReader_18(Build) 3323.33 root data:Selection_17 +│ └─Selection_17 3323.33 cop[tikv] lt(planner__core__casetest__integration.t1.b, 10) +│ └─TableFullScan_16 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─IndexLookUp_12(Probe) 4154.17 root partition:all + ├─IndexRangeScan_9(Build) 12500.00 cop[tikv] table:t2_part, index:a(a) range: decided by [eq(planner__core__casetest__integration.t2_part.a, planner__core__casetest__integration.t1.a)], keep order:false, stats:pseudo + └─Selection_11(Probe) 4154.17 cop[tikv] lt(planner__core__casetest__integration.t2_part.b, 20) + └─TableRowIDScan_10 12500.00 cop[tikv] table:t2_part keep order:false, stats:pseudo +set @@tidb_skip_missing_partition_stats = default; +drop table if exists t; +create table t(a int(11) not null, b int) partition by range (a) (partition p0 values less than (4), partition p1 values less than(10), partition p2 values less than maxvalue); +insert into t values (1, 1),(10, 10),(11, 11); +set tidb_opt_fix_control='44262:ON'; +explain format = 'brief' select * from t where a in (1, 2,'11'); +id estRows task access object operator info +TableReader 30.00 root partition:p0,p2 data:Selection +└─Selection 30.00 cop[tikv] in(planner__core__casetest__integration.t.a, 1, 2, 11) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where a in (17, null); +id estRows task access object operator info +TableReader 10.00 root partition:p0,p2 data:Selection +└─Selection 10.00 cop[tikv] in(planner__core__casetest__integration.t.a, 17, NULL) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where a in (16, 'abc'); +id estRows task access object operator info +TableReader 20.00 root partition:p0,p2 data:Selection +└─Selection 20.00 cop[tikv] in(planner__core__casetest__integration.t.a, 16, 0) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where a in (15, 0.12, 3.47); +id estRows task access object operator info +TableReader 10.00 root partition:p2 data:Selection +└─Selection 10.00 cop[tikv] or(eq(planner__core__casetest__integration.t.a, 15), 0) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where a in (0.12, 3.47); +id estRows task access object operator info +TableDual 0.00 root rows:0 +explain format = 'brief' select * from t where a in (14, floor(3.47)); +id estRows task access object operator info +TableReader 20.00 root partition:p0,p2 data:Selection +└─Selection 20.00 cop[tikv] in(planner__core__casetest__integration.t.a, 14, 3) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where b in (3, 4); +id estRows task access object operator info +TableReader 20.00 root partition:all data:Selection +└─Selection 20.00 cop[tikv] in(planner__core__casetest__integration.t.b, 3, 4) + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +set tidb_opt_fix_control=default; +drop table if exists pt; +create table pt (id int, c int, key i_id(id), key i_c(c)) partition by range (c) ( +partition p0 values less than (4), +partition p1 values less than (7), +partition p2 values less than (10)); +set @@tidb_enable_index_merge = 1; +set tidb_opt_fix_control='44262:ON'; +## Table reader +explain format='brief' select * from pt where c > 10; +id estRows task access object operator info +TableReader 3333.33 root partition:dual data:Selection +└─Selection 3333.33 cop[tikv] gt(planner__core__casetest__integration.pt.c, 10) + └─TableFullScan 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo +explain format='brief' select * from pt where c > 8; +id estRows task access object operator info +TableReader 3333.33 root partition:p2 data:Selection +└─Selection 3333.33 cop[tikv] gt(planner__core__casetest__integration.pt.c, 8) + └─TableFullScan 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo +explain format='brief' select * from pt where c < 2 or c >= 9; +id estRows task access object operator info +TableReader 6656.67 root partition:p0,p2 data:Selection +└─Selection 6656.67 cop[tikv] or(lt(planner__core__casetest__integration.pt.c, 2), ge(planner__core__casetest__integration.pt.c, 9)) + └─TableFullScan 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo +## Index reader +explain format='brief' select c from pt; +id estRows task access object operator info +IndexReader 10000.00 root partition:all index:IndexFullScan +└─IndexFullScan 10000.00 cop[tikv] table:pt, index:i_c(c) keep order:false, stats:pseudo +explain format='brief' select c from pt where c > 10; +id estRows task access object operator info +IndexReader 3333.33 root partition:dual index:IndexRangeScan +└─IndexRangeScan 3333.33 cop[tikv] table:pt, index:i_c(c) range:(10,+inf], keep order:false, stats:pseudo +explain format='brief' select c from pt where c > 8; +id estRows task access object operator info +IndexReader 3333.33 root partition:p2 index:IndexRangeScan +└─IndexRangeScan 3333.33 cop[tikv] table:pt, index:i_c(c) range:(8,+inf], keep order:false, stats:pseudo +explain format='brief' select c from pt where c < 2 or c >= 9; +id estRows task access object operator info +IndexReader 6656.67 root partition:p0,p2 index:IndexRangeScan +└─IndexRangeScan 6656.67 cop[tikv] table:pt, index:i_c(c) range:[-inf,2), [9,+inf], keep order:false, stats:pseudo +## Index Lookup +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt; +id estRows task access object operator info +IndexLookUp 10000.00 root partition:all +├─IndexFullScan(Build) 10000.00 cop[tikv] table:pt, index:i_id(id) keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt where id < 4 and c > 10; +id estRows task access object operator info +IndexLookUp 1107.78 root partition:dual +├─IndexRangeScan(Build) 3323.33 cop[tikv] table:pt, index:i_id(id) range:[-inf,4), keep order:false, stats:pseudo +└─Selection(Probe) 1107.78 cop[tikv] gt(planner__core__casetest__integration.pt.c, 10) + └─TableRowIDScan 3323.33 cop[tikv] table:pt keep order:false, stats:pseudo +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c > 8; +id estRows task access object operator info +IndexLookUp 1107.78 root partition:p2 +├─IndexRangeScan(Build) 3323.33 cop[tikv] table:pt, index:i_id(id) range:[-inf,10), keep order:false, stats:pseudo +└─Selection(Probe) 1107.78 cop[tikv] gt(planner__core__casetest__integration.pt.c, 8) + └─TableRowIDScan 3323.33 cop[tikv] table:pt keep order:false, stats:pseudo +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt where id < 10 and c < 2 or c >= 9; +id estRows task access object operator info +IndexLookUp 5325.33 root partition:p0,p2 +├─IndexFullScan(Build) 10000.00 cop[tikv] table:pt, index:i_id(id) keep order:false, stats:pseudo +└─Selection(Probe) 5325.33 cop[tikv] or(and(lt(planner__core__casetest__integration.pt.id, 10), lt(planner__core__casetest__integration.pt.c, 2)), ge(planner__core__casetest__integration.pt.c, 9)) + └─TableRowIDScan 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo +## Partition selection +explain format='brief' select * from pt partition (p0) where c > 8; +id estRows task access object operator info +TableReader 3333.33 root partition:dual data:Selection +└─Selection 3333.33 cop[tikv] gt(planner__core__casetest__integration.pt.c, 8) + └─TableFullScan 10000.00 cop[tikv] table:pt keep order:false, stats:pseudo +explain format='brief' select c from pt partition (p0, p2) where c > 8; +id estRows task access object operator info +IndexReader 3333.33 root partition:p2 index:IndexRangeScan +└─IndexRangeScan 3333.33 cop[tikv] table:pt, index:i_c(c) range:(8,+inf], keep order:false, stats:pseudo +explain format='brief' select /*+ use_index(pt, i_id) */ * from pt partition (p1, p2) where c < 3 and id = 5; +id estRows task access object operator info +IndexLookUp 3.32 root partition:dual +├─IndexRangeScan(Build) 10.00 cop[tikv] table:pt, index:i_id(id) range:[5,5], keep order:false, stats:pseudo +└─Selection(Probe) 3.32 cop[tikv] lt(planner__core__casetest__integration.pt.c, 3) + └─TableRowIDScan 10.00 cop[tikv] table:pt keep order:false, stats:pseudo +## Index Merge +explain format='brief' select * from pt where id = 4 or c < 7; +id estRows task access object operator info +IndexMerge 3330.01 root partition:all type: union +├─IndexRangeScan(Build) 10.00 cop[tikv] table:pt, index:i_id(id) range:[4,4], keep order:false, stats:pseudo +├─IndexRangeScan(Build) 3323.33 cop[tikv] table:pt, index:i_c(c) range:[-inf,7), keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 3330.01 cop[tikv] table:pt keep order:false, stats:pseudo +explain format='brief' select * from pt where id > 4 or c = 7; +id estRows task access object operator info +IndexMerge 3340.00 root partition:all type: union +├─IndexRangeScan(Build) 3333.33 cop[tikv] table:pt, index:i_id(id) range:(4,+inf], keep order:false, stats:pseudo +├─IndexRangeScan(Build) 10.00 cop[tikv] table:pt, index:i_c(c) range:[7,7], keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 3340.00 cop[tikv] table:pt keep order:false, stats:pseudo +set tidb_opt_fix_control=default; +set @@tidb_enable_index_merge = default; +drop table if exists github_events; +CREATE TABLE `github_events` ( +`id` bigint(20) NOT NULL DEFAULT '0', +`type` varchar(29) NOT NULL DEFAULT 'Event', +`created_at` datetime NOT NULL DEFAULT '1970-01-01 00:00:00', +`repo_id` bigint(20) NOT NULL DEFAULT '0', +`repo_name` varchar(140) NOT NULL DEFAULT '', +`actor_id` bigint(20) NOT NULL DEFAULT '0', +`actor_login` varchar(40) NOT NULL DEFAULT '', +`language` varchar(26) NOT NULL DEFAULT '', +`additions` bigint(20) NOT NULL DEFAULT '0', +`deletions` bigint(20) NOT NULL DEFAULT '0', +`action` varchar(11) NOT NULL DEFAULT '', +`number` int(11) NOT NULL DEFAULT '0', +`commit_id` varchar(40) NOT NULL DEFAULT '', +`comment_id` bigint(20) NOT NULL DEFAULT '0', +`org_login` varchar(40) NOT NULL DEFAULT '', +`org_id` bigint(20) NOT NULL DEFAULT '0', +`state` varchar(6) NOT NULL DEFAULT '', +`closed_at` datetime NOT NULL DEFAULT '1970-01-01 00:00:00', +`comments` int(11) NOT NULL DEFAULT '0', +`pr_merged_at` datetime NOT NULL DEFAULT '1970-01-01 00:00:00', +`pr_merged` tinyint(1) NOT NULL DEFAULT '0', +`pr_changed_files` int(11) NOT NULL DEFAULT '0', +`pr_review_comments` int(11) NOT NULL DEFAULT '0', +`pr_or_issue_id` bigint(20) NOT NULL DEFAULT '0', +`event_day` date NOT NULL, +`event_month` date NOT NULL, +`event_year` int(11) NOT NULL, +`push_size` int(11) NOT NULL DEFAULT '0', +`push_distinct_size` int(11) NOT NULL DEFAULT '0', +`creator_user_login` varchar(40) NOT NULL DEFAULT '', +`creator_user_id` bigint(20) NOT NULL DEFAULT '0', +`pr_or_issue_created_at` datetime NOT NULL DEFAULT '1970-01-01 00:00:00', +KEY `index_github_events_on_id` (`id`), +KEY `index_github_events_on_created_at` (`created_at`), +KEY `index_github_events_on_repo_id_type_action_month_actor_login` (`repo_id`,`type`,`action`,`event_month`,`actor_login`), +KEY `index_ge_on_repo_id_type_action_pr_merged_created_at_add_del` (`repo_id`,`type`,`action`,`pr_merged`,`created_at`,`additions`,`deletions`), +KEY `index_ge_on_creator_id_type_action_merged_created_at_add_del` (`creator_user_id`,`type`,`action`,`pr_merged`,`created_at`,`additions`,`deletions`), +KEY `index_ge_on_actor_id_type_action_created_at_repo_id_commits` (`actor_id`,`type`,`action`,`created_at`,`repo_id`,`push_distinct_size`), +KEY `index_ge_on_repo_id_type_action_created_at_number_pdsize_psize` (`repo_id`,`type`,`action`,`created_at`,`number`,`push_distinct_size`,`push_size`), +KEY `index_ge_on_repo_id_type_action_created_at_actor_login` (`repo_id`,`type`,`action`,`created_at`,`actor_login`), +KEY `index_ge_on_repo_name_type` (`repo_name`,`type`), +KEY `index_ge_on_actor_login_type` (`actor_login`,`type`), +KEY `index_ge_on_org_login_type` (`org_login`,`type`), +KEY `index_ge_on_language` (`language`), +KEY `index_ge_on_org_id_type` (`org_id`,`type`), +KEY `index_ge_on_actor_login_lower` ((lower(`actor_login`))), +KEY `index_ge_on_repo_name_lower` ((lower(`repo_name`))), +KEY `index_ge_on_language_lower` ((lower(`language`))), +KEY `index_ge_on_type_action` (`type`,`action`) /*!80000 INVISIBLE */, +KEY `index_ge_on_repo_id_type_created_at` (`repo_id`,`type`,`created_at`), +KEY `index_ge_on_repo_id_created_at` (`repo_id`,`created_at`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY LIST COLUMNS(`type`) +(PARTITION `push_event` VALUES IN ('PushEvent'), +PARTITION `create_event` VALUES IN ('CreateEvent'), +PARTITION `pull_request_event` VALUES IN ('PullRequestEvent'), +PARTITION `watch_event` VALUES IN ('WatchEvent'), +PARTITION `issue_comment_event` VALUES IN ('IssueCommentEvent'), +PARTITION `issues_event` VALUES IN ('IssuesEvent'), +PARTITION `delete_event` VALUES IN ('DeleteEvent'), +PARTITION `fork_event` VALUES IN ('ForkEvent'), +PARTITION `pull_request_review_comment_event` VALUES IN ('PullRequestReviewCommentEvent'), +PARTITION `pull_request_review_event` VALUES IN ('PullRequestReviewEvent'), +PARTITION `gollum_event` VALUES IN ('GollumEvent'), +PARTITION `release_event` VALUES IN ('ReleaseEvent'), +PARTITION `member_event` VALUES IN ('MemberEvent'), +PARTITION `commit_comment_event` VALUES IN ('CommitCommentEvent'), +PARTITION `public_event` VALUES IN ('PublicEvent'), +PARTITION `gist_event` VALUES IN ('GistEvent'), +PARTITION `follow_event` VALUES IN ('FollowEvent'), +PARTITION `event` VALUES IN ('Event'), +PARTITION `download_event` VALUES IN ('DownloadEvent'), +PARTITION `team_add_event` VALUES IN ('TeamAddEvent'), +PARTITION `fork_apply_event` VALUES IN ('ForkApplyEvent')); +SELECT +repo_id, GROUP_CONCAT( +DISTINCT actor_login +ORDER BY cnt DESC +SEPARATOR ',' +) AS actor_logins +FROM ( +SELECT +ge.repo_id AS repo_id, +ge.actor_login AS actor_login, +COUNT(*) AS cnt +FROM github_events ge +WHERE +type = 'PullRequestEvent' AND action = 'opened' +AND (ge.created_at >= DATE_SUB(NOW(), INTERVAL 1 DAY) AND ge.created_at <= NOW()) +GROUP BY ge.repo_id, ge.actor_login +ORDER BY cnt DESC +) sub +GROUP BY repo_id; +repo_id actor_logins +drop database if exists testdb; +create database testdb; +use testdb; +drop table if exists `t270`; +CREATE TABLE `t270` ( +`vkey` int(11) DEFAULT NULL, +`pkey` int(11) DEFAULT NULL, +`c1128` varchar(100) DEFAULT NULL, +`c1129` int(11) DEFAULT NULL, +`c1130` varchar(100) DEFAULT NULL, +`c1131` double DEFAULT NULL, +`c1132` varchar(100) DEFAULT NULL, +`c1133` double DEFAULT NULL, +`c1134` varchar(100) DEFAULT NULL, +`c1135` int(11) DEFAULT NULL +); +drop table if exists `t271`; +CREATE TABLE `t271` ( +`vkey` int(11) DEFAULT NULL, +`pkey` int(11) DEFAULT NULL, +`c1136` varchar(100) DEFAULT NULL, +`c1137` int(11) DEFAULT NULL, +`c1138` varchar(100) DEFAULT NULL, +`c1139` int(11) DEFAULT NULL, +`c1140` double DEFAULT NULL, +`c1141` int(11) DEFAULT NULL +); +drop table if exists `t272`; +CREATE TABLE `t272` ( +`vkey` int(11) DEFAULT NULL, +`pkey` int(11) DEFAULT NULL, +`c1142` int(11) DEFAULT NULL, +`c1143` varchar(100) DEFAULT NULL, +`c1144` int(11) DEFAULT NULL, +`c1145` int(11) DEFAULT NULL, +`c1146` varchar(100) DEFAULT NULL, +`c1147` double DEFAULT NULL, +`c1148` varchar(100) DEFAULT NULL, +`c1149` double DEFAULT NULL +); +CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`%` SQL SECURITY DEFINER VIEW `t273_test` (`c0`, `c1`, `c2`, `c3`, `c4`) AS SELECT AVG(37) OVER (PARTITION BY `ref_0`.`c1136` ORDER BY `ref_0`.`vkey` DESC,`ref_0`.`pkey` DESC,`ref_0`.`c1136` DESC,`ref_0`.`c1137`,`ref_0`.`c1138` DESC,`ref_0`.`c1139` DESC,`ref_0`.`c1140` DESC,`ref_0`.`c1141`) AS `c0`,COALESCE(`ref_0`.`c1137`, `ref_0`.`c1141`) AS `c1`,`ref_0`.`vkey` AS `c2`,`ref_0`.`pkey` AS `c3`,`ref_0`.`c1138` AS `c4` FROM `testdb`.`t271` AS `ref_0` WHERE EXISTS (SELECT `subq_0`.`c2` AS `c0`,`subq_0`.`c0` AS `c1`,`subq_0`.`c0` AS `c2`,`subq_0`.`c0` AS `c3`,CASE WHEN EXISTS (SELECT `ref_9`.`c1131` AS `c0`,`ref_9`.`c1131` AS `c1`,(FALSE) XOR (((-45)=(-69)) OR ((-0)>(-71))) AS `c2`,`ref_9`.`c1133` AS `c3`,`ref_9`.`c1128` AS `c4`,-0 AS `c5`,1 AS `c6`,`ref_9`.`c1132` AS `c7`,`ref_9`.`c1131` AS `c8`,`ref_9`.`c1130` AS `c9`,NULL AS `c10` FROM `testdb`.`t270` AS `ref_9` WHERE (-0)<(-8) UNION ALL SELECT `ref_0`.`c1140` AS `c0`,`ref_11`.`c1133` AS `c1`,(NULL)<(NULL) AS `c2`,`ref_0`.`c1140` AS `c3`,`ref_0`.`c1136` AS `c4`,95 AS `c5`,NOT (_UTF8MB4'mum#M' LIKE _UTF8MB4'%_U') AS `c6`,`ref_11`.`c1128` AS `c7`,`ref_11`.`c1131` AS `c8`,(SELECT `c1143` AS `c1143` FROM `testdb`.`t272` ORDER BY `c1143` LIMIT 3,1) AS `c9`,97 AS `c10` FROM `testdb`.`t270` AS `ref_11` WHERE NOT (TRUE)) THEN _UTF8MB4'xf' ELSE _UTF8MB4'>c' END LIKE _UTF8MB4'_^^' AS `c4`,`subq_0`.`c1` AS `c5`,`ref_0`.`vkey` AS `c6`,((`subq_0`.`c1`)=(SELECT `ref_12`.`c1132` AS `c0` FROM `testdb`.`t270` AS `ref_12` WHERE TRUE ORDER BY `c0` DESC LIMIT 1)) XOR ((`ref_0`.`pkey`)>=(SELECT (SELECT `vkey` AS `vkey` FROM `testdb`.`t271` ORDER BY `vkey` LIMIT 1,1) AS `c0` FROM `testdb`.`t271` AS `ref_13` WHERE (-24)<=((SELECT COUNT(`c1140`) AS `count(c1140)` FROM `testdb`.`t271`)) ORDER BY `c0` LIMIT 1)) AS `c7`,`ref_0`.`pkey` AS `c8`,`subq_0`.`c2` AS `c9`,`ref_0`.`vkey` AS `c10`,`ref_0`.`c1139` AS `c11`,TRUE AS `c12`,`subq_0`.`c0` AS `c13`,`subq_0`.`c2` AS `c14`,`subq_0`.`c2` AS `c15`,FALSE AS `c16`,CASE WHEN ((FALSE) OR ((((FALSE) XOR (((-73)<(-91)) OR (((-0) BETWEEN (-0) AND (-0)) AND ((NULL) OR ((0)>((SELECT COUNT(`c1131`) AS `count(c1131)` FROM `testdb`.`t270`))))))) AND ((-19)>(NULL))) OR (((77)<(73)) AND (NOT (((73) IN (SELECT 0 AS `c0` FROM `testdb`.`t271` AS `ref_14` WHERE (NULL) AND (NULL) EXCEPT SELECT NULL AS `c0` FROM `testdb`.`t270` AS `ref_15` WHERE (`ref_15`.`c1131`)!=(SELECT `ref_15`.`c1133` AS `c0` FROM `testdb`.`t270` AS `ref_16` WHERE _UTF8MB4'$@-X' LIKE _UTF8MB4'__%' ORDER BY `c0` DESC LIMIT 1))) IS TRUE))))) OR (NOT ((-24)<=(-43))) THEN `subq_0`.`c1` ELSE `subq_0`.`c2` END AS `c17`,`subq_0`.`c1` AS `c18`,`subq_0`.`c0` AS `c19`,`subq_0`.`c0` AS `c20`,`subq_0`.`c2` AS `c21`,`subq_0`.`c0` AS `c22`,`subq_0`.`c2` AS `c23`,`subq_0`.`c0` AS `c24`,`ref_0`.`c1141` AS `c25` FROM (SELECT DISTINCT TRUE AS `c0`,`ref_1`.`c1143` AS `c1`,`ref_1`.`c1146` AS `c2` FROM `testdb`.`t272` AS `ref_1` WHERE NOT (((`ref_0`.`c1136`)!=(SELECT `ref_2`.`c1146` AS `c0` FROM `testdb`.`t272` AS `ref_2` WHERE (62) BETWEEN ((SELECT COUNT(`c1147`) AS `count(c1147)` FROM `testdb`.`t272`)) AND (-0) ORDER BY `c0` LIMIT 1)) XOR ((-0) BETWEEN (0) AND (-0)))) AS `subq_0` WHERE (CHAR_LENGTH(CASE WHEN ((`subq_0`.`c0`) IS NOT NULL) OR ((`ref_0`.`c1138`)>(SELECT `ref_0`.`c1138` AS `c0` FROM `testdb`.`t272` AS `ref_3` WHERE FALSE ORDER BY `c0` DESC LIMIT 1)) THEN _UTF8MB4'' ELSE _UTF8MB4'tL' END)) BETWEEN (ABS(46%-11)) AND (CASE WHEN (((((`subq_0`.`c2`) IN (SELECT `ref_4`.`c1134` AS `c0` FROM `testdb`.`t270` AS `ref_4` WHERE (NULL LIKE _UTF8MB4'%Ny') OR (EXISTS (SELECT DISTINCT `ref_5`.`c1136` AS `c0`,`ref_5`.`c1140` AS `c1` FROM `testdb`.`t271` AS `ref_5` WHERE FALSE UNION ALL SELECT `ref_4`.`c1130` AS `c0`,`ref_4`.`c1131` AS `c1` FROM `testdb`.`t271` AS `ref_6` WHERE (-97) BETWEEN (73) AND (-10))) UNION ALL SELECT `ref_7`.`c1138` AS `c0` FROM `testdb`.`t271` AS `ref_7` WHERE FALSE)) IS TRUE) OR (NULL)) AND ((NULL)>=((SELECT COUNT(`c1140`) AS `count(c1140)` FROM `testdb`.`t271`)))) XOR (((`ref_0`.`vkey`) IN (SELECT `ref_8`.`c1145` AS `c0` FROM `testdb`.`t272` AS `ref_8` WHERE ((FALSE) AND (NULL)) OR ((`ref_8`.`c1144`) IS NULL))) IS TRUE) THEN 87 ELSE CASE WHEN ((`ref_0`.`c1138`) IS NULL) OR ((-22)!=(-0)) THEN 17 ELSE -67 END END)) ORDER BY `c0` DESC,`c1` DESC,`c2`,`c3`,`c4` DESC; +select +(select +subq_1.c0 as c0 +from +t273_test as ref_84 +where exists ( +select +(select +ref_86.c1147 as c0 +from +t272 as ref_86 +where (subq_1.c0) > (subq_1.c0) +window w0 as (partition by ref_86.c1147 order by ref_86.c1143 desc) +order by c0 limit 1 +) as c3, +(select +subq_1.c0 as c0 +from +t273_test as ref_89 +order by c0 limit 1) as c4 +from +t271 as ref_85 +) +order by c0 desc limit 1) as c1 +from +(select 1 as c0) as subq_1; +c1 +NULL +select +(select +subq_1.c0 as c0 +from +t271 as ref_84 +where exists ( +select +(select +ref_86.c1147 as c0 +from +t272 as ref_86 +where (subq_1.c0) > (subq_1.c0) +window w0 as (partition by ref_86.c1147 order by ref_86.c1143 desc) +order by c0 limit 1 +) as c3, +(select +subq_1.c0 as c0 +from +t271 as ref_89 +order by c0 limit 1) as c4 +from +t271 as ref_85 +) +order by c0 desc limit 1) as c1 +from +(select 1 as c0) as subq_1; +c1 +NULL +use test; +drop table if exists tt; +create table tt(a bigint, b bigint, c bigint, d bigint, e bigint, primary key(c,d)); +explain format = brief update tt, (select 1 as c1 ,2 as c2 ,3 as c3, 4 as c4 union all select 2,3,4,5 union all select 3,4,5,6) tmp set tt.a=tmp.c1, tt.b=tmp.c2 where tt.c=tmp.c3 and tt.d=tmp.c4 and (tt.c,tt.d) in ((11,111),(22,222),(33,333),(44,444)); +id estRows task access object operator info +Update N/A root N/A +└─Projection 0.00 root test.tt.a, test.tt.b, test.tt.c, test.tt.d, test.tt.e, Column#18, Column#19, Column#20, Column#21 + └─IndexJoin 0.00 root inner join, inner:TableReader, outer key:Column#20, Column#21, inner key:test.tt.c, test.tt.d, equal cond:eq(Column#20, test.tt.c), eq(Column#21, test.tt.d), other cond:or(or(and(eq(Column#20, 11), eq(test.tt.d, 111)), and(eq(Column#20, 22), eq(test.tt.d, 222))), or(and(eq(Column#20, 33), eq(test.tt.d, 333)), and(eq(Column#20, 44), eq(test.tt.d, 444)))), or(or(and(eq(test.tt.c, 11), eq(Column#21, 111)), and(eq(test.tt.c, 22), eq(Column#21, 222))), or(and(eq(test.tt.c, 33), eq(Column#21, 333)), and(eq(test.tt.c, 44), eq(Column#21, 444)))) + ├─Union(Build) 0.00 root + │ ├─Projection 0.00 root 1->Column#18, 2->Column#19, 3->Column#20, 4->Column#21 + │ │ └─TableDual 0.00 root rows:0 + │ ├─Projection 0.00 root 2->Column#18, 3->Column#19, 4->Column#20, 5->Column#21 + │ │ └─TableDual 0.00 root rows:0 + │ └─Projection 0.00 root 3->Column#18, 4->Column#19, 5->Column#20, 6->Column#21 + │ └─TableDual 0.00 root rows:0 + └─TableReader(Probe) 0.00 root data:Selection + └─Selection 0.00 cop[tikv] or(or(and(eq(test.tt.c, 11), eq(test.tt.d, 111)), and(eq(test.tt.c, 22), eq(test.tt.d, 222))), or(and(eq(test.tt.c, 33), eq(test.tt.d, 333)), and(eq(test.tt.c, 44), eq(test.tt.d, 444)))), or(or(eq(test.tt.c, 11), eq(test.tt.c, 22)), or(eq(test.tt.c, 33), eq(test.tt.c, 44))), or(or(eq(test.tt.d, 111), eq(test.tt.d, 222)), or(eq(test.tt.d, 333), eq(test.tt.d, 444))) + └─TableRangeScan 0.00 cop[tikv] table:tt range: decided by [eq(test.tt.c, Column#20) eq(test.tt.d, Column#21)], keep order:false, stats:pseudo 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..039d92a3891d8 --- /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 all columns; +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 all columns; +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 all columns; +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 all columns; +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 1.00 root partition:2023p1 index:Selection_9 +└─Selection_9 1.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 1.00 root partition:2023p2 index:Selection_9 +└─Selection_9 1.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 1.00 root partition:2023p1 index:Selection_9 +└─Selection_9 1.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 1.00 root partition:2023p2 index:Selection_9 +└─Selection_9 1.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/tests/integrationtest/r/statistics/integration.result b/tests/integrationtest/r/statistics/integration.result new file mode 100644 index 0000000000000..4baea6abdeeb1 --- /dev/null +++ b/tests/integrationtest/r/statistics/integration.result @@ -0,0 +1,51 @@ +drop table if exists t; +set @@session.tidb_analyze_version = 2; +create table t(a longtext); +insert into t value(repeat("a",65536)); +insert into t value(repeat("b",65536)); +analyze table t with 0 topn; +set @@session.tidb_analyze_version = default; +drop table if exists t1; +drop table if exists t2; +create table t1(a int); +create table t2(a int); +insert into t1 values(null), (null), (null), (null); +insert into t2 values(null), (null); +analyze table t1 all columns; +analyze table t2 all columns; +explain format = 'brief' select * from t1 left join t2 on t1.a=t2.a order by t1.a, t2.a; +id estRows task access object operator info +Sort 4.00 root statistics__integration.t1.a, statistics__integration.t2.a +└─HashJoin 4.00 root left outer join, equal:[eq(statistics__integration.t1.a, statistics__integration.t2.a)] + ├─TableReader(Build) 1.00 root data:Selection + │ └─Selection 1.00 cop[tikv] not(isnull(statistics__integration.t2.a)) + │ └─TableFullScan 2.00 cop[tikv] table:t2 keep order:false + └─TableReader(Probe) 4.00 root data:TableFullScan + └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false +explain format = 'brief' select * from t2 left join t1 on t1.a=t2.a order by t1.a, t2.a; +id estRows task access object operator info +Sort 2.00 root statistics__integration.t1.a, statistics__integration.t2.a +└─HashJoin 2.00 root left outer join, equal:[eq(statistics__integration.t2.a, statistics__integration.t1.a)] + ├─TableReader(Build) 1.00 root data:Selection + │ └─Selection 1.00 cop[tikv] not(isnull(statistics__integration.t1.a)) + │ └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false + └─TableReader(Probe) 2.00 root data:TableFullScan + └─TableFullScan 2.00 cop[tikv] table:t2 keep order:false +explain format = 'brief' select * from t1 right join t2 on t1.a=t2.a order by t1.a, t2.a; +id estRows task access object operator info +Sort 2.00 root statistics__integration.t1.a, statistics__integration.t2.a +└─HashJoin 2.00 root right outer join, equal:[eq(statistics__integration.t1.a, statistics__integration.t2.a)] + ├─TableReader(Build) 1.00 root data:Selection + │ └─Selection 1.00 cop[tikv] not(isnull(statistics__integration.t1.a)) + │ └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false + └─TableReader(Probe) 2.00 root data:TableFullScan + └─TableFullScan 2.00 cop[tikv] table:t2 keep order:false +explain format = 'brief' select * from t2 right join t1 on t1.a=t2.a order by t1.a, t2.a; +id estRows task access object operator info +Sort 4.00 root statistics__integration.t1.a, statistics__integration.t2.a +└─HashJoin 4.00 root right outer join, equal:[eq(statistics__integration.t2.a, statistics__integration.t1.a)] + ├─TableReader(Build) 1.00 root data:Selection + │ └─Selection 1.00 cop[tikv] not(isnull(statistics__integration.t2.a)) + │ └─TableFullScan 2.00 cop[tikv] table:t2 keep order:false + └─TableReader(Probe) 4.00 root data:TableFullScan + └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false diff --git a/tests/integrationtest/r/util/ranger.result b/tests/integrationtest/r/util/ranger.result new file mode 100644 index 0000000000000..c6cfef64a3bc3 --- /dev/null +++ b/tests/integrationtest/r/util/ranger.result @@ -0,0 +1,588 @@ +drop table if exists t; +create table t(a varchar(100), b int, c int, index idx_ab(a, b)); +insert into t values ('kk', 1, 10), ('kk', 1, 20), ('hh', 2, 10), ('hh', 3, 10), ('xx', 4, 10), ('yy', 5, 10), ('yy', 6, 20), ('zz', 7, 10); +set @@tidb_opt_fix_control = '44389:ON'; +explain format='brief' select * from t where c = 10 and (a = 'xx' or (a = 'kk' and b = 1)); +id estRows task access object operator info +IndexLookUp 0.01 root +├─IndexRangeScan(Build) 10.10 cop[tikv] table:t, index:idx_ab(a, b) range:["kk" 1,"kk" 1], ["xx","xx"], keep order:false, stats:pseudo +└─Selection(Probe) 0.01 cop[tikv] eq(util__ranger.t.c, 10) + └─TableRowIDScan 10.10 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where c = 10 and (a = 'xx' or (a = 'kk' and b = 1)); +a b c +kk 1 10 +xx 4 10 +explain format='brief' select * from t where c = 10 and ((a = 'xx' or a = 'yy') or ((a = 'kk' and b = 1) or (a = 'hh' and b = 2))); +id estRows task access object operator info +IndexLookUp 0.02 root +├─IndexRangeScan(Build) 20.20 cop[tikv] table:t, index:idx_ab(a, b) range:["hh" 2,"hh" 2], ["kk" 1,"kk" 1], ["xx","xx"], ["yy","yy"], keep order:false, stats:pseudo +└─Selection(Probe) 0.02 cop[tikv] eq(util__ranger.t.c, 10) + └─TableRowIDScan 20.20 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where c = 10 and ((a = 'xx' or a = 'yy') or ((a = 'kk' and b = 1) or (a = 'hh' and b = 2))); +a b c +hh 2 10 +kk 1 10 +xx 4 10 +yy 5 10 +set @@tidb_opt_fix_control = default; +DROP TABLE IF EXISTS IDT_20755; +CREATE TABLE `IDT_20755` ( +`COL1` varchar(20) DEFAULT NULL, +`COL2` tinyint(16) DEFAULT NULL, +`COL3` timestamp NULL DEFAULT NULL, +KEY `u_m_col` (`COL1`(10),`COL2`,`COL3`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +INSERT INTO IDT_20755 VALUES("牾窓螎刳闌蜹瑦詬鍖湪槢壿玟瞏膍敗特森撇縆", 73, "2010-06-03 07:29:05"); +INSERT INTO IDT_20755 VALUES("xxxxxxxxxxxxxxx", 73, "2010-06-03 07:29:05"); +explain format = 'brief' select * from IDT_20755 use index (u_m_col) where col1 in ("牾窓螎刳闌蜹瑦詬鍖湪槢壿玟瞏膍敗特森撇縆", "物碃貞枕騫摨聫嚣蜻禼担堋黕詖蝒毎槒阆畒郒", "剮毵樍穋摻瀽鬦擀钟鷫產冖悄乮曙枱诠鑡轰砠") and col2 in (72, 39, 73) and col3 != "2024-10-19 08:55:32"; +id estRows task access object operator info +IndexLookUp 5.99 root +├─IndexRangeScan(Build) 5.99 cop[tikv] table:IDT_20755, index:u_m_col(COL1, COL2, COL3) range:["剮毵樍穋摻瀽鬦擀钟鷫" 39 -inf,"剮毵樍穋摻瀽鬦擀钟鷫" 39 2024-10-19 08:55:32), ("剮毵樍穋摻瀽鬦擀钟鷫" 39 2024-10-19 08:55:32,"剮毵樍穋摻瀽鬦擀钟鷫" 39 +inf], ["剮毵樍穋摻瀽鬦擀钟鷫" 72 -inf,"剮毵樍穋摻瀽鬦擀钟鷫" 72 2024-10-19 08:55:32), ("剮毵樍穋摻瀽鬦擀钟鷫" 72 2024-10-19 08:55:32,"剮毵樍穋摻瀽鬦擀钟鷫" 72 +inf], ["剮毵樍穋摻瀽鬦擀钟鷫" 73 -inf,"剮毵樍穋摻瀽鬦擀钟鷫" 73 2024-10-19 08:55:32), ("剮毵樍穋摻瀽鬦擀钟鷫" 73 2024-10-19 08:55:32,"剮毵樍穋摻瀽鬦擀钟鷫" 73 +inf], ["物碃貞枕騫摨聫嚣蜻禼" 39 -inf,"物碃貞枕騫摨聫嚣蜻禼" 39 2024-10-19 08:55:32), ("物碃貞枕騫摨聫嚣蜻禼" 39 2024-10-19 08:55:32,"物碃貞枕騫摨聫嚣蜻禼" 39 +inf], ["物碃貞枕騫摨聫嚣蜻禼" 72 -inf,"物碃貞枕騫摨聫嚣蜻禼" 72 2024-10-19 08:55:32), ("物碃貞枕騫摨聫嚣蜻禼" 72 2024-10-19 08:55:32,"物碃貞枕騫摨聫嚣蜻禼" 72 +inf], ["物碃貞枕騫摨聫嚣蜻禼" 73 -inf,"物碃貞枕騫摨聫嚣蜻禼" 73 2024-10-19 08:55:32), ("物碃貞枕騫摨聫嚣蜻禼" 73 2024-10-19 08:55:32,"物碃貞枕騫摨聫嚣蜻禼" 73 +inf], ["牾窓螎刳闌蜹瑦詬鍖湪" 39 -inf,"牾窓螎刳闌蜹瑦詬鍖湪" 39 2024-10-19 08:55:32), ("牾窓螎刳闌蜹瑦詬鍖湪" 39 2024-10-19 08:55:32,"牾窓螎刳闌蜹瑦詬鍖湪" 39 +inf], ["牾窓螎刳闌蜹瑦詬鍖湪" 72 -inf,"牾窓螎刳闌蜹瑦詬鍖湪" 72 2024-10-19 08:55:32), ("牾窓螎刳闌蜹瑦詬鍖湪" 72 2024-10-19 08:55:32,"牾窓螎刳闌蜹瑦詬鍖湪" 72 +inf], ["牾窓螎刳闌蜹瑦詬鍖湪" 73 -inf,"牾窓螎刳闌蜹瑦詬鍖湪" 73 2024-10-19 08:55:32), ("牾窓螎刳闌蜹瑦詬鍖湪" 73 2024-10-19 08:55:32,"牾窓螎刳闌蜹瑦詬鍖湪" 73 +inf], keep order:false, stats:pseudo +└─Selection(Probe) 5.99 cop[tikv] in(util__ranger.idt_20755.col1, "牾窓螎刳闌蜹瑦詬鍖湪槢壿玟瞏膍敗特森撇縆", "物碃貞枕騫摨聫嚣蜻禼担堋黕詖蝒毎槒阆畒郒", "剮毵樍穋摻瀽鬦擀钟鷫產冖悄乮曙枱诠鑡轰砠") + └─TableRowIDScan 5.99 cop[tikv] table:IDT_20755 keep order:false, stats:pseudo +select * from IDT_20755 use index (u_m_col) where col1 in ("牾窓螎刳闌蜹瑦詬鍖湪槢壿玟瞏膍敗特森撇縆", "物碃貞枕騫摨聫嚣蜻禼担堋黕詖蝒毎槒阆畒郒", "剮毵樍穋摻瀽鬦擀钟鷫產冖悄乮曙枱诠鑡轰砠") and col2 in (72, 39, 73) and col3 != "2024-10-19 08:55:32"; +COL1 COL2 COL3 +牾窓螎刳闌蜹瑦詬鍖湪槢壿玟瞏膍敗特森撇縆 73 2010-06-03 07:29:05 +explain format = 'brief' select * from IDT_20755 use index (u_m_col) where col1 = "xxxxxxxxxxxxxxx" and col2 in (72, 73) and col3 != "2024-10-19 08:55:32"; +id estRows task access object operator info +IndexLookUp 1.33 root +├─IndexRangeScan(Build) 1.33 cop[tikv] table:IDT_20755, index:u_m_col(COL1, COL2, COL3) range:["xxxxxxxxxx" 72 -inf,"xxxxxxxxxx" 72 2024-10-19 08:55:32), ("xxxxxxxxxx" 72 2024-10-19 08:55:32,"xxxxxxxxxx" 72 +inf], ["xxxxxxxxxx" 73 -inf,"xxxxxxxxxx" 73 2024-10-19 08:55:32), ("xxxxxxxxxx" 73 2024-10-19 08:55:32,"xxxxxxxxxx" 73 +inf], keep order:false, stats:pseudo +└─Selection(Probe) 1.33 cop[tikv] eq(util__ranger.idt_20755.col1, "xxxxxxxxxxxxxxx") + └─TableRowIDScan 1.33 cop[tikv] table:IDT_20755 keep order:false, stats:pseudo +select * from IDT_20755 use index (u_m_col) where col1 = "xxxxxxxxxxxxxxx" and col2 in (72, 73) and col3 != "2024-10-19 08:55:32"; +COL1 COL2 COL3 +xxxxxxxxxxxxxxx 73 2010-06-03 07:29:05 +explain format = 'brief' select * from IDT_20755 use index (u_m_col) where col1 = "xxxxxxxxxxxxxxx" and col2 in (72, 73, 74) and col3 != "2024-10-19 08:55:32"; +id estRows task access object operator info +IndexLookUp 2.00 root +├─IndexRangeScan(Build) 2.00 cop[tikv] table:IDT_20755, index:u_m_col(COL1, COL2, COL3) range:["xxxxxxxxxx" 72 -inf,"xxxxxxxxxx" 72 2024-10-19 08:55:32), ("xxxxxxxxxx" 72 2024-10-19 08:55:32,"xxxxxxxxxx" 72 +inf], ["xxxxxxxxxx" 73 -inf,"xxxxxxxxxx" 73 2024-10-19 08:55:32), ("xxxxxxxxxx" 73 2024-10-19 08:55:32,"xxxxxxxxxx" 73 +inf], ["xxxxxxxxxx" 74 -inf,"xxxxxxxxxx" 74 2024-10-19 08:55:32), ("xxxxxxxxxx" 74 2024-10-19 08:55:32,"xxxxxxxxxx" 74 +inf], keep order:false, stats:pseudo +└─Selection(Probe) 2.00 cop[tikv] eq(util__ranger.idt_20755.col1, "xxxxxxxxxxxxxxx") + └─TableRowIDScan 2.00 cop[tikv] table:IDT_20755 keep order:false, stats:pseudo +select * from IDT_20755 use index (u_m_col) where col1 = "xxxxxxxxxxxxxxx" and col2 in (72, 73, 74) and col3 != "2024-10-19 08:55:32"; +COL1 COL2 COL3 +xxxxxxxxxxxxxxx 73 2010-06-03 07:29:05 +drop table if exists t1, t2; +create table t1(a decimal unsigned, key(a)); +insert into t1 values(0),(null); +create table t2(a int, b decimal unsigned, key idx(a,b)); +insert into t2 values(1,0),(1,null); +explain format = 'brief' select * from t1 use index(a) where a in (-1,0); +id estRows task access object operator info +IndexReader 10.00 root index:IndexRangeScan +└─IndexRangeScan 10.00 cop[tikv] table:t1, index:a(a) range:[0,0], keep order:false, stats:pseudo +select * from t1 use index(a) where a in (-1,0); +a +0 +explain format = 'brief' select * from t1 use index(a) where a = -1; +id estRows task access object operator info +TableDual 0.00 root rows:0 +select * from t1 use index(a) where a = -1; +a +explain format = 'brief' select * from t1 use index(a) where a > -1; +id estRows task access object operator info +IndexReader 3333.33 root index:IndexRangeScan +└─IndexRangeScan 3333.33 cop[tikv] table:t1, index:a(a) range:[0,+inf], keep order:false, stats:pseudo +select * from t1 use index(a) where a > -1; +a +0 +explain format = 'brief' select * from t1 use index(a) where a < -1; +id estRows task access object operator info +TableDual 0.00 root rows:0 +select * from t1 use index(a) where a < -1; +a +explain format = 'brief' select * from t1 use index(a) where a <= -1; +id estRows task access object operator info +TableDual 0.00 root rows:0 +select * from t1 use index(a) where a <= -1; +a +explain format = 'brief' select * from t1 use index(a) where a >= -1; +id estRows task access object operator info +IndexReader 3333.33 root index:IndexRangeScan +└─IndexRangeScan 3333.33 cop[tikv] table:t1, index:a(a) range:[0,+inf], keep order:false, stats:pseudo +select * from t1 use index(a) where a >= -1; +a +0 +explain format = 'brief' select * from t2 use index(idx) where a = 1 and b in (-1,0); +id estRows task access object operator info +IndexReader 0.10 root index:IndexRangeScan +└─IndexRangeScan 0.10 cop[tikv] table:t2, index:idx(a, b) range:[1 0,1 0], keep order:false, stats:pseudo +select * from t2 use index(idx) where a = 1 and b in (-1,0); +a b +1 0 +explain format = 'brief' select * from t2 use index(idx) where a = 1 and b = -1; +id estRows task access object operator info +TableDual 0.00 root rows:0 +select * from t2 use index(idx) where a = 1 and b = -1; +a b +explain format = 'brief' select * from t2 use index(idx) where a = 1 and b > -1; +id estRows task access object operator info +IndexReader 33.33 root index:IndexRangeScan +└─IndexRangeScan 33.33 cop[tikv] table:t2, index:idx(a, b) range:[1 0,1 +inf], keep order:false, stats:pseudo +select * from t2 use index(idx) where a = 1 and b > -1; +a b +1 0 +explain format = 'brief' select * from t2 use index(idx) where a = 1 and b < -1; +id estRows task access object operator info +TableDual 0.00 root rows:0 +select * from t2 use index(idx) where a = 1 and b < -1; +a b +explain format = 'brief' select * from t2 use index(idx) where a = 1 and b <= -1; +id estRows task access object operator info +TableDual 0.00 root rows:0 +select * from t2 use index(idx) where a = 1 and b <= -1; +a b +explain format = 'brief' select * from t2 use index(idx) where a = 1 and b >= -1; +id estRows task access object operator info +IndexReader 33.33 root index:IndexRangeScan +└─IndexRangeScan 33.33 cop[tikv] table:t2, index:idx(a, b) range:[1 0,1 +inf], keep order:false, stats:pseudo +select * from t2 use index(idx) where a = 1 and b >= -1; +a b +1 0 +set @@tidb_partition_prune_mode = 'static'; +set @@tidb_executor_concurrency = 1; +drop table if exists t; +CREATE TABLE `t` (a bit(1) DEFAULT NULL,b int(11) DEFAULT NULL) PARTITION BY HASH(a)PARTITIONS 3; +insert ignore into t values(-1, -1), (0, 0), (1, 1), (3, 3); +analyze table t all columns; +explain format='brief' select * from t; +id estRows task access object operator info +PartitionUnion 4.00 root +├─TableReader 1.00 root data:TableFullScan +│ └─TableFullScan 1.00 cop[tikv] table:t, partition:p0 keep order:false +└─TableReader 3.00 root data:TableFullScan + └─TableFullScan 3.00 cop[tikv] table:t, partition:p1 keep order:false +select * from t; +a b + 0 + -1 + 1 + 3 +explain format='brief' select * from t where a = 0; +id estRows task access object operator info +TableReader 1.00 root data:Selection +└─Selection 1.00 cop[tikv] eq(util__ranger.t.a, 0) + └─TableFullScan 1.00 cop[tikv] table:t, partition:p0 keep order:false +select * from t where a = 0; +a b + 0 +explain format='brief' select * from t where a = 0 or a = 4; +id estRows task access object operator info +TableReader 1.00 root data:Selection +└─Selection 1.00 cop[tikv] or(eq(util__ranger.t.a, 0), eq(util__ranger.t.a, 4)) + └─TableFullScan 1.00 cop[tikv] table:t, partition:p0 keep order:false +select * from t where a = 0 or a = 4; +a b + 0 +explain format='brief' select * from t where a = 1; +id estRows task access object operator info +TableReader 3.00 root data:Selection +└─Selection 3.00 cop[tikv] eq(util__ranger.t.a, 1) + └─TableFullScan 3.00 cop[tikv] table:t, partition:p1 keep order:false +select * from t where a = 1; +a b + -1 + 1 + 3 +explain format='brief' select * from t where a = -1; +id estRows task access object operator info +TableDual 0.00 root rows:0 +select * from t where a = -1; +a b +explain format='brief' select * from t where a = 3; +id estRows task access object operator info +TableDual 0.00 root rows:0 +select * from t where a = 3; +a b +explain format='brief' select * from t where a < 1; +id estRows task access object operator info +PartitionUnion 2.00 root +├─TableReader 1.00 root data:Selection +│ └─Selection 1.00 cop[tikv] lt(util__ranger.t.a, 1) +│ └─TableFullScan 1.00 cop[tikv] table:t, partition:p0 keep order:false +└─TableReader 1.00 root data:Selection + └─Selection 1.00 cop[tikv] lt(util__ranger.t.a, 1) + └─TableFullScan 3.00 cop[tikv] table:t, partition:p1 keep order:false +select * from t where a < 1; +a b + 0 +explain format='brief' select * from t where a < 3; +id estRows task access object operator info +PartitionUnion 4.00 root +├─TableReader 1.00 root data:Selection +│ └─Selection 1.00 cop[tikv] lt(util__ranger.t.a, 3) +│ └─TableFullScan 1.00 cop[tikv] table:t, partition:p0 keep order:false +└─TableReader 3.00 root data:Selection + └─Selection 3.00 cop[tikv] lt(util__ranger.t.a, 3) + └─TableFullScan 3.00 cop[tikv] table:t, partition:p1 keep order:false +select * from t where a < 3; +a b + 0 + -1 + 1 + 3 +explain format='brief' select * from t where a < -1; +id estRows task access object operator info +TableDual 0.00 root rows:0 +select * from t where a < -1; +a b +explain format='brief' select * from t where a > 0; +id estRows task access object operator info +PartitionUnion 4.00 root +├─TableReader 1.00 root data:Selection +│ └─Selection 1.00 cop[tikv] gt(util__ranger.t.a, 0) +│ └─TableFullScan 1.00 cop[tikv] table:t, partition:p0 keep order:false +└─TableReader 3.00 root data:Selection + └─Selection 3.00 cop[tikv] gt(util__ranger.t.a, 0) + └─TableFullScan 3.00 cop[tikv] table:t, partition:p1 keep order:false +select * from t where a > 0; +a b + -1 + 1 + 3 +explain format='brief' select * from t where a > -1; +id estRows task access object operator info +PartitionUnion 4.00 root +├─TableReader 1.00 root data:Selection +│ └─Selection 1.00 cop[tikv] gt(util__ranger.t.a, -1) +│ └─TableFullScan 1.00 cop[tikv] table:t, partition:p0 keep order:false +└─TableReader 3.00 root data:Selection + └─Selection 3.00 cop[tikv] gt(util__ranger.t.a, -1) + └─TableFullScan 3.00 cop[tikv] table:t, partition:p1 keep order:false +select * from t where a > -1; +a b + 0 + -1 + 1 + 3 +explain format='brief' select * from t where a > 3; +id estRows task access object operator info +PartitionUnion 2.00 root +├─TableReader 1.00 root data:Selection +│ └─Selection 1.00 cop[tikv] gt(util__ranger.t.a, 3) +│ └─TableFullScan 1.00 cop[tikv] table:t, partition:p0 keep order:false +└─TableReader 1.00 root data:Selection + └─Selection 1.00 cop[tikv] gt(util__ranger.t.a, 3) + └─TableFullScan 3.00 cop[tikv] table:t, partition:p1 keep order:false +select * from t where a > 3; +a b +set @@tidb_partition_prune_mode = default; +set @@tidb_executor_concurrency = default; +drop table if exists t2; +create table t2 (id int unsigned not null auto_increment primary key, t text, index(t(3))); +insert into t2 (t) values ('aaaa'),('a'); +explain format='brief' select * from t2 where t='aaaa'; +id estRows task access object operator info +IndexLookUp 10.00 root +├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:t(t) range:["aaa","aaa"], keep order:false, stats:pseudo +└─Selection(Probe) 10.00 cop[tikv] eq(util__ranger.t2.t, "aaaa") + └─TableRowIDScan 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo +select * from t2 where t='aaaa'; +id t +1 aaaa +explain format='brief' select * from t2 where t='aaaa' or t = 'a'; +id estRows task access object operator info +IndexLookUp 16.00 root +├─IndexRangeScan(Build) 20.00 cop[tikv] table:t2, index:t(t) range:["a","a"], ["aaa","aaa"], keep order:false, stats:pseudo +└─Selection(Probe) 16.00 cop[tikv] or(eq(util__ranger.t2.t, "aaaa"), eq(util__ranger.t2.t, "a")) + └─TableRowIDScan 20.00 cop[tikv] table:t2 keep order:false, stats:pseudo +select * from t2 where t='aaaa' or t = 'a'; +id t +1 aaaa +2 a +analyze table t2; +explain format='brief' select * from t2 where t='aaaa'; +id estRows task access object operator info +TableReader 1.00 root data:Selection +└─Selection 1.00 cop[tikv] eq(util__ranger.t2.t, "aaaa") + └─TableFullScan 2.00 cop[tikv] table:t2 keep order:false +select * from t2 where t='aaaa'; +id t +1 aaaa +explain format='brief' select * from t2 where t='aaaa' or t = 'a'; +id estRows task access object operator info +TableReader 1.60 root data:Selection +└─Selection 1.60 cop[tikv] or(eq(util__ranger.t2.t, "aaaa"), eq(util__ranger.t2.t, "a")) + └─TableFullScan 2.00 cop[tikv] table:t2 keep order:false +select * from t2 where t='aaaa' or t = 'a'; +id t +1 aaaa +2 a +drop table if exists t; +create table t(a varchar(100), b int, c int, d int, index idx(a, b, c)); +insert into t values ('t',1,1,1),('t',1,3,3),('t',2,1,3),('t',2,3,1),('w',0,3,3),('z',0,1,1); +explain format='brief' select * from t use index (idx) where ((a = 't' and b = 1) or (a = 't' and b = 2) or (a = 'w' and b = 0)) and c > 2; +id estRows task access object operator info +IndexLookUp 1.00 root +├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:idx(a, b, c) range:("t" 1 2,"t" 1 +inf], ("t" 2 2,"t" 2 +inf], ("w" 0 2,"w" 0 +inf], keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (idx) where ((a = 't' and b = 1) or (a = 't' and b = 2) or (a = 'w' and b = 0)) and c > 2; +a b c d +t 1 3 3 +t 2 3 1 +w 0 3 3 +explain format='brief' select * from t use index (idx) where ((a = 't' and b = 1) or (a = 't' and b = 2) or (a = 'w' and b = 0)) and d > 2; +id estRows task access object operator info +IndexLookUp 0.10 root +├─IndexRangeScan(Build) 0.30 cop[tikv] table:t, index:idx(a, b, c) range:["t" 1,"t" 1], ["t" 2,"t" 2], ["w" 0,"w" 0], keep order:false, stats:pseudo +└─Selection(Probe) 0.10 cop[tikv] gt(util__ranger.t.d, 2) + └─TableRowIDScan 0.30 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (idx) where ((a = 't' and b = 1) or (a = 't' and b = 2) or (a = 'w' and b = 0)) and d > 2; +a b c d +t 1 3 3 +t 2 1 3 +w 0 3 3 +drop table if exists t; +create table t(a int, b int, c int, primary key(a,b,c)); +insert into t values(1,1,1),(2,2,3); +analyze table t; +explain format='brief' select * from t where a = 1 and (b,c) in ((1,1),(2,3)); +id estRows task access object operator info +Batch_Point_Get 2.00 root table:t, index:PRIMARY(a, b, c) keep order:false, desc:false +select * from t where a = 1 and (b,c) in ((1,1),(2,3)); +a b c +1 1 1 +explain format='brief' select * from t where a = 1 and ((b = 1 and c = 1) or (b = 2 and c = 3)); +id estRows task access object operator info +Batch_Point_Get 2.00 root table:t, index:PRIMARY(a, b, c) keep order:false, desc:false +select * from t where a = 1 and ((b = 1 and c = 1) or (b = 2 and c = 3)); +a b c +1 1 1 +explain format='brief' select * from t where a = 1 and ((b = 1) or (b = 2 and c = 3)); +id estRows task access object operator info +IndexReader 2.00 root index:IndexRangeScan +└─IndexRangeScan 2.00 cop[tikv] table:t, index:PRIMARY(a, b, c) range:[1 1,1 1], [1 2 3,1 2 3], keep order:false +select * from t where a = 1 and ((b = 1) or (b = 2 and c = 3)); +a b c +1 1 1 +explain format='brief' select * from t where (a,b) in ((1,1),(2,2)) and c = 3; +id estRows task access object operator info +Batch_Point_Get 2.00 root table:t, index:PRIMARY(a, b, c) keep order:false, desc:false +select * from t where (a,b) in ((1,1),(2,2)) and c = 3; +a b c +2 2 3 +explain format='brief' select * from t where ((a = 1 and b = 1) or (a = 2 and b = 2)) and c = 3; +id estRows task access object operator info +Batch_Point_Get 2.00 root table:t, index:PRIMARY(a, b, c) keep order:false, desc:false +select * from t where ((a = 1 and b = 1) or (a = 2 and b = 2)) and c = 3; +a b c +2 2 3 +explain format='brief' select * from t use index(primary) where ((a = 1) or (a = 2 and b = 2)) and c = 3; +id estRows task access object operator info +IndexReader 1.00 root index:Selection +└─Selection 1.00 cop[tikv] eq(util__ranger.t.c, 3) + └─IndexRangeScan 2.00 cop[tikv] table:t, index:PRIMARY(a, b, c) range:[1,1], [2 2,2 2], keep order:false +select * from t use index(primary) where ((a = 1) or (a = 2 and b = 2)) and c = 3; +a b c +2 2 3 +explain format='brief' select * from t where (a,b) in ((1,1),(2,2)) and c > 2 and (a,b,c) in ((1,1,1),(2,2,3)); +id estRows task access object operator info +Selection 0.75 root gt(util__ranger.t.c, 2), or(and(eq(util__ranger.t.a, 1), eq(util__ranger.t.b, 1)), and(eq(util__ranger.t.a, 2), eq(util__ranger.t.b, 2))) +└─Batch_Point_Get 2.00 root table:t, index:PRIMARY(a, b, c) keep order:false, desc:false +select * from t where (a,b) in ((1,1),(2,2)) and c > 2 and (a,b,c) in ((1,1,1),(2,2,3)); +a b c +2 2 3 +explain format='brief' select * from t where (a,b) in ((1,1),(2,2)) and c > 2; +id estRows task access object operator info +IndexReader 2.00 root index:IndexRangeScan +└─IndexRangeScan 2.00 cop[tikv] table:t, index:PRIMARY(a, b, c) range:(1 1 2,1 1 +inf], (2 2 2,2 2 +inf], keep order:false +select * from t where (a,b) in ((1,1),(2,2)) and c > 2; +a b c +2 2 3 +explain format='brief' select * from t where ((a = 1 and b = 1) or (a = 2 and b = 2)) and c > 2; +id estRows task access object operator info +IndexReader 2.00 root index:IndexRangeScan +└─IndexRangeScan 2.00 cop[tikv] table:t, index:PRIMARY(a, b, c) range:(1 1 2,1 1 +inf], (2 2 2,2 2 +inf], keep order:false +select * from t where ((a = 1 and b = 1) or (a = 2 and b = 2)) and c > 2; +a b c +2 2 3 +drop table if exists t; +create table t(a int, b int, c int, primary key(a,b)); +insert into t values(1,1,1),(2,2,3); +analyze table t all columns; +explain format='brief' select * from t where (a,b) in ((1,1),(2,2)) and c = 3; +id estRows task access object operator info +Selection 1.00 root eq(util__ranger.t.c, 3) +└─Batch_Point_Get 2.00 root table:t, index:PRIMARY(a, b) keep order:false, desc:false +select * from t where (a,b) in ((1,1),(2,2)) and c = 3; +a b c +2 2 3 +explain format='brief' select * from t where ((a = 1 and b = 1) or (a = 2 and b = 2)) and c = 3; +id estRows task access object operator info +Selection 1.00 root eq(util__ranger.t.c, 3) +└─Batch_Point_Get 2.00 root table:t, index:PRIMARY(a, b) keep order:false, desc:false +select * from t where ((a = 1 and b = 1) or (a = 2 and b = 2)) and c = 3; +a b c +2 2 3 +explain format='brief' select * from t use index(primary) where ((a = 1) or (a = 2 and b = 2)) and c = 3; +id estRows task access object operator info +IndexLookUp 1.00 root +├─IndexRangeScan(Build) 2.00 cop[tikv] table:t, index:PRIMARY(a, b) range:[1,1], [2 2,2 2], keep order:false +└─Selection(Probe) 1.00 cop[tikv] eq(util__ranger.t.c, 3) + └─TableRowIDScan 2.00 cop[tikv] table:t keep order:false +select * from t use index(primary) where ((a = 1) or (a = 2 and b = 2)) and c = 3; +a b c +2 2 3 +explain format='brief' select * from t where ((a = 1 and b = 1) or (a = 2 and b = 2)) and c = 3 and (a = 1 or a = 2); +id estRows task access object operator info +Selection 1.00 root eq(util__ranger.t.c, 3), or(eq(util__ranger.t.a, 1), eq(util__ranger.t.a, 2)) +└─Batch_Point_Get 2.00 root table:t, index:PRIMARY(a, b) keep order:false, desc:false +select * from t where ((a = 1 and b = 1) or (a = 2 and b = 2)) and c = 3 and (a = 1 or a = 2); +a b c +2 2 3 +explain format='brief' select * from t where (a,b) in ((1,1),(2,2)) and c > 2; +id estRows task access object operator info +Selection 1.00 root gt(util__ranger.t.c, 2) +└─Batch_Point_Get 2.00 root table:t, index:PRIMARY(a, b) keep order:false, desc:false +select * from t where (a,b) in ((1,1),(2,2)) and c > 2; +a b c +2 2 3 +explain format='brief' select * from t where ((a = 1 and b = 1) or (a = 2 and b = 2)) and c > 2; +id estRows task access object operator info +Selection 1.00 root gt(util__ranger.t.c, 2) +└─Batch_Point_Get 2.00 root table:t, index:PRIMARY(a, b) keep order:false, desc:false +select * from t where ((a = 1 and b = 1) or (a = 2 and b = 2)) and c > 2; +a b c +2 2 3 +set @@session.tidb_regard_null_as_point=false; +drop table if exists t; +create table t(a int, b int, c int, key(a,b,c)); +insert into t values(1,2,2); +explain format='brief' select * from t where a = 1 and b in (1, 2) and c > 1; +id estRows task access object operator info +IndexReader 0.67 root index:IndexRangeScan +└─IndexRangeScan 0.67 cop[tikv] table:t, index:a(a, b, c) range:(1 1 1,1 1 +inf], (1 2 1,1 2 +inf], keep order:false, stats:pseudo +select * from t where a = 1 and b in (1, 2) and c > 1; +a b c +1 2 2 +explain format='brief' select * from t where a = 1 and (b = 1 or b = 2) and c > 1; +id estRows task access object operator info +IndexReader 0.67 root index:IndexRangeScan +└─IndexRangeScan 0.67 cop[tikv] table:t, index:a(a, b, c) range:(1 1 1,1 1 +inf], (1 2 1,1 2 +inf], keep order:false, stats:pseudo +select * from t where a = 1 and (b = 1 or b = 2) and c > 1; +a b c +1 2 2 +explain format='brief' select * from t where a = 1 and (b = 1 or b in (2, 3)) and c > 1; +id estRows task access object operator info +IndexReader 1.00 root index:IndexRangeScan +└─IndexRangeScan 1.00 cop[tikv] table:t, index:a(a, b, c) range:(1 1 1,1 1 +inf], (1 2 1,1 2 +inf], (1 3 1,1 3 +inf], keep order:false, stats:pseudo +select * from t where a = 1 and (b = 1 or b in (2, 3)) and c > 1; +a b c +1 2 2 +explain format='brief' select * from t where a = 1 and (b = 1 or b = 2) and b = 3 and c > 1; +id estRows task access object operator info +TableDual 0.00 root rows:0 +select * from t where a = 1 and (b = 1 or b = 2) and b = 3 and c > 1; +a b c +explain format='brief' select * from t where a = 1 and (b is null or b = 2); +id estRows task access object operator info +IndexReader 0.20 root index:IndexRangeScan +└─IndexRangeScan 0.20 cop[tikv] table:t, index:a(a, b, c) range:[1 NULL,1 NULL], [1 2,1 2], keep order:false, stats:pseudo +select * from t where a = 1 and (b is null or b = 2); +a b c +1 2 2 +explain format='brief' select * from t where a = 1 and (b is null or b = 2) and c > 1; +id estRows task access object operator info +IndexReader 0.67 root index:IndexRangeScan +└─IndexRangeScan 0.67 cop[tikv] table:t, index:a(a, b, c) range:(1 NULL 1,1 NULL +inf], (1 2 1,1 2 +inf], keep order:false, stats:pseudo +select * from t where a = 1 and (b is null or b = 2) and c > 1; +a b c +1 2 2 +explain format='brief' select * from t where a = 1 and b is null and c > 1; +id estRows task access object operator info +IndexReader 0.03 root index:Selection +└─Selection 0.03 cop[tikv] gt(util__ranger.t.c, 1) + └─IndexRangeScan 0.10 cop[tikv] table:t, index:a(a, b, c) range:[1 NULL,1 NULL], keep order:false, stats:pseudo +select * from t where a = 1 and b is null and c > 1; +a b c +explain format='brief' select * from t where a = 1 and b is null and b is null and c > 1; +id estRows task access object operator info +IndexReader 0.03 root index:Selection +└─Selection 0.03 cop[tikv] gt(util__ranger.t.c, 1) + └─IndexRangeScan 0.10 cop[tikv] table:t, index:a(a, b, c) range:[1 NULL,1 NULL], keep order:false, stats:pseudo +select * from t where a = 1 and b is null and b is null and c > 1; +a b c +explain format='brief' select * from t where a = 1 and b is null and b = 1 and c > 1; +id estRows task access object operator info +TableDual 0.00 root rows:0 +select * from t where a = 1 and b is null and b = 1 and c > 1; +a b c +set @@session.tidb_regard_null_as_point=default; +set tidb_cost_model_version=2; +drop table if exists t0; +CREATE TABLE t0(c0 TEXT(10)); +INSERT INTO t0(c0) VALUES (1); +CREATE INDEX i0 ON t0(c0(255)); +analyze table t0; +explain format = 'brief' select * from t0 where c0; +id estRows task access object operator info +IndexReader 1.00 root index:Selection +└─Selection 1.00 cop[tikv] util__ranger.t0.c0 + └─IndexFullScan 1.00 cop[tikv] table:t0, index:i0(c0) keep order:false +explain format = 'brief' select * from t0 where c0 and c0 > '123'; +id estRows task access object operator info +IndexReader 1.00 root index:Selection +└─Selection 1.00 cop[tikv] util__ranger.t0.c0 + └─IndexRangeScan 1.00 cop[tikv] table:t0, index:i0(c0) range:("123",+inf], keep order:false +explain format = 'brief' select * from t0 where c0 and c0 <> '123'; +id estRows task access object operator info +IndexReader 1.00 root index:Selection +└─Selection 1.00 cop[tikv] util__ranger.t0.c0 + └─IndexRangeScan 1.00 cop[tikv] table:t0, index:i0(c0) range:[-inf,"123"), ("123",+inf], keep order:false +explain format = 'brief' select * from t0 where c0 is true; +id estRows task access object operator info +IndexReader 1.00 root index:Selection +└─Selection 1.00 cop[tikv] istrue(cast(util__ranger.t0.c0, double BINARY)) + └─IndexFullScan 1.00 cop[tikv] table:t0, index:i0(c0) keep order:false +explain format = 'brief' select * from t0 where c0 is false; +id estRows task access object operator info +IndexReader 1.00 root index:Selection +└─Selection 1.00 cop[tikv] isfalse(cast(util__ranger.t0.c0, double BINARY)) + └─IndexFullScan 1.00 cop[tikv] table:t0, index:i0(c0) keep order:false +explain format = 'brief' select * from t0 where c0 and c0 in ('123','456','789'); +id estRows task access object operator info +IndexReader 1.00 root index:Selection +└─Selection 1.00 cop[tikv] util__ranger.t0.c0 + └─IndexRangeScan 1.00 cop[tikv] table:t0, index:i0(c0) range:["123","123"], ["456","456"], ["789","789"], keep order:false +explain format = 'brief' select * FROM t0 WHERE ('a' != t0.c0) AND t0.c0; +id estRows task access object operator info +IndexReader 1.00 root index:Selection +└─Selection 1.00 cop[tikv] util__ranger.t0.c0 + └─IndexRangeScan 1.00 cop[tikv] table:t0, index:i0(c0) range:[-inf,"a"), ("a",+inf], keep order:false +set tidb_cost_model_version=default; +set tidb_cost_model_version=2; +drop table if exists t; +create table t(a int primary key, b int, c int, d int, e int, index idx(b,c,d)); +insert into t values(1,1,1,1,2),(2,1,2,1,0); +analyze table t; +explain format = 'brief' select t.e in (select count(*) from t s use index(idx), t t1 where s.b = 1 and s.c in (1, 2) and s.d = t.a and s.a = t1.a) from t; +id estRows task access object operator info +Projection 2.00 root Column#27 +└─Apply 2.00 root CARTESIAN left outer semi join, other cond:eq(util__ranger.t.e, Column#26) + ├─TableReader(Build) 2.00 root data:TableFullScan + │ └─TableFullScan 2.00 cop[tikv] table:t keep order:false + └─StreamAgg(Probe) 2.00 root funcs:count(1)->Column#26 + └─HashJoin 4.00 root inner join, equal:[eq(util__ranger.t.a, util__ranger.t.a)] + ├─IndexReader(Build) 4.00 root index:IndexFullScan + │ └─IndexFullScan 4.00 cop[tikv] table:t1, index:idx(b, c, d) keep order:false + └─IndexReader(Probe) 4.00 root index:IndexRangeScan + └─IndexRangeScan 4.00 cop[tikv] table:s, index:idx(b, c, d) range: decided by [eq(util__ranger.t.b, 1) in(util__ranger.t.c, 1, 2) eq(util__ranger.t.d, util__ranger.t.a)], keep order:false +select t.e in (select count(*) from t s use index(idx), t t1 where s.b = 1 and s.c in (1, 2) and s.d = t.a and s.a = t1.a) from t; +t.e in (select count(*) from t s use index(idx), t t1 where s.b = 1 and s.c in (1, 2) and s.d = t.a and s.a = t1.a) +1 +1 +set tidb_cost_model_version=default; +drop table if exists t; +create table t(a int not null, b int not null, primary key(a,b)); +insert into t values(1,2); +analyze table t; +explain format = 'brief' select * from (select * from t union all select a, b from t) sub where a > 0; +id estRows task access object operator info +Union 2.00 root +├─IndexReader 1.00 root index:IndexRangeScan +│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:PRIMARY(a, b) range:(0,+inf], keep order:false +└─IndexReader 1.00 root index:IndexRangeScan + └─IndexRangeScan 1.00 cop[tikv] table:t, index:PRIMARY(a, b) range:(0,+inf], keep order:false +select * from (select * from t union all select ifnull(a,b), b from t) sub where a > 0; +a b +1 2 +1 2 diff --git a/tests/integrationtest/t/planner/cardinality/selectivity.test b/tests/integrationtest/t/planner/cardinality/selectivity.test new file mode 100644 index 0000000000000..b865738ef56bb --- /dev/null +++ b/tests/integrationtest/t/planner/cardinality/selectivity.test @@ -0,0 +1,703 @@ +# TestPrimaryKeySelectivity +drop table if exists t; +set tidb_enable_clustered_index = 'INT_ONLY'; +create table t(a char(10) primary key, b int); +explain select * from t where a > "t"; +drop table t; +create table t(a int primary key, b int); +explain select * from t where a > 1; +set tidb_enable_clustered_index = DEFAULT; +drop table t; + +# TestStatsVer2 +set tidb_cost_model_version=2; +set tidb_analyze_version=2; +drop table if exists tint; +create table tint(a int, b int, c int, index singular(a), index multi(b, c)); +insert into tint values (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5), (6, 6, 6), (7, 7, 7), (8, 8, 8); +analyze table tint all columns with 2 topn, 3 buckets; +drop table if exists tdouble; +create table tdouble(a double, b double, c double, index singular(a), index multi(b, c)); +insert into tdouble values (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5), (6, 6, 6), (7, 7, 7), (8, 8, 8); +analyze table tdouble all columns with 2 topn, 3 buckets; +drop table if exists tdecimal; +create table tdecimal(a decimal(40, 20), b decimal(40, 20), c decimal(40, 20), index singular(a), index multi(b, c)); +insert into tdecimal values (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5), (6, 6, 6), (7, 7, 7), (8, 8, 8); +analyze table tdecimal all columns with 2 topn, 3 buckets; +drop table if exists tstring; +create table tstring(a varchar(64), b varchar(64), c varchar(64), index singular(a), index multi(b, c)); +insert into tstring values ('1', '1', '1'), ('2', '2', '2'), ('3', '3', '3'), ('4', '4', '4'), ('5', '5', '5'), ('6', '6', '6'), ('7', '7', '7'), ('8', '8', '8'); +analyze table tstring all columns with 2 topn, 3 buckets; +drop table if exists tdatetime; +create table tdatetime(a datetime, b datetime, c datetime, index singular(a), index multi(b, c)); +insert into tdatetime values ('2001-01-01', '2001-01-01', '2001-01-01'), ('2001-01-02', '2001-01-02', '2001-01-02'), ('2001-01-03', '2001-01-03', '2001-01-03'), ('2001-01-04', '2001-01-04', '2001-01-04'); +analyze table tdatetime all columns with 2 topn, 3 buckets; +drop table if exists tprefix; +create table tprefix(a varchar(64), b varchar(64), index prefixa(a(2))); +insert into tprefix values ('111', '111'), ('222', '222'), ('333', '333'), ('444', '444'), ('555', '555'), ('666', '666'); +analyze table tprefix all columns with 2 topn, 3 buckets; +drop table if exists ct1; +create table ct1 (a int, pk varchar(10), primary key(pk) clustered); +insert into ct1 values (1, '1'), (2, '2'), (3, '3'), (4, '4'), (5, '5'), (6, '6'), (7, '7'), (8, '8'); +analyze table ct1 all columns with 2 topn, 3 buckets; +drop table if exists ct2; +create table ct2 (a int, b int, c int, primary key(a, b) clustered); +insert into ct2 values (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5), (6, 6, 6), (7, 7, 7), (8, 8, 8); +analyze table ct2 all columns with 2 topn, 3 buckets; +--sorted_result +select stats_ver from mysql.stats_histograms where table_id in (select TIDB_TABLE_ID from information_schema.tables where TABLE_SCHEMA = 'planner__cardinality__selectivity'); +--sorted_result +show stats_topn where db_name = 'planner__cardinality__selectivity'; +--sorted_result +show stats_buckets where db_name = 'planner__cardinality__selectivity'; +explain select * from tint where a=1; +explain select * from tint where a=4; +explain select * from tint where a=8; +explain select * from tdouble where a=1; +explain select * from tdouble where a=4; +explain select * from tdouble where a=8; +explain select * from tdecimal where a=1; +explain select * from tdecimal where a=4; +explain select * from tdecimal where a=8; +explain select * from tstring where a='1'; +explain select * from tstring where a='4'; +explain select * from tstring where a='8'; +explain select * from tdatetime where a='2001-01-01'; +explain select * from tdatetime where a='2001-01-02'; +explain select * from tdatetime where a='2001-01-04'; +explain select * from tprefix where a='111'; +explain select * from tprefix where a='444'; +explain select * from tprefix where a='888'; +explain select * from tint where b=1 and c=1; +explain select * from tint where b=4 and c=4; +explain select * from tint where b=8 and c=8; +explain select * from tdouble where b=1 and c=1; +explain select * from tdouble where b=4 and c=4; +explain select * from tdouble where b=8 and c=8; +explain select * from tdecimal where b=1 and c=1; +explain select * from tdecimal where b=4 and c=4; +explain select * from tdecimal where b=8 and c=8; +explain select * from tstring where b='1' and c='1'; +explain select * from tstring where b='4' and c='4'; +explain select * from tstring where b='8' and c='8'; +explain select * from tdatetime where b='2001-01-01' and c='2001-01-01'; +explain select * from tdatetime where b='2001-01-02' and c='2001-01-02'; +explain select * from tdatetime where b='2001-01-04' and c='2001-01-04'; +explain select * from tint where b=1; +explain select * from tint where b=4; +explain select * from tint where b=8; +explain select * from tdouble where b=1; +explain select * from tdouble where b=4; +explain select * from tdouble where b=8; +explain select * from tdecimal where b=1; +explain select * from tdecimal where b=4; +explain select * from tdecimal where b=8; +explain select * from tstring where b='1'; +explain select * from tstring where b='4'; +explain select * from tstring where b='8'; +explain select * from tdatetime where b='2001-01-01'; +explain select * from tdatetime where b='2001-01-02'; +explain select * from tdatetime where b='2001-01-04'; +explain select * from ct1 where pk>='1' and pk <='4'; +explain select * from ct1 where pk>='4' and pk <='6'; +explain select * from ct1 where pk>='6' and pk <='8'; +explain select * from ct2 where a=1 and b>=1 and b<=8; +explain select * from ct2 where a=4 and b>=1 and b<=8; +explain select * from ct2 where a=8 and b>=1 and b<=8; + +# TestTopNOutOfHist +set tidb_analyze_version=2; +drop table if exists topn_before_hist; +create table topn_before_hist(a int, index idx(a)); +insert into topn_before_hist values(1), (1), (1), (1), (3), (3), (4), (5), (6); +analyze table topn_before_hist all columns with 2 topn, 3 buckets; +create table topn_after_hist(a int, index idx(a)); +insert into topn_after_hist values(2), (2), (3), (4), (5), (7), (7), (7), (7); +analyze table topn_after_hist all columns with 2 topn, 3 buckets; +create table topn_before_hist_no_index(a int); +insert into topn_before_hist_no_index values(1), (1), (1), (1), (3), (3), (4), (5), (6); +analyze table topn_before_hist_no_index all columns with 2 topn, 3 buckets; +create table topn_after_hist_no_index(a int); +insert into topn_after_hist_no_index values(2), (2), (3), (4), (5), (7), (7), (7), (7); +analyze table topn_after_hist_no_index all columns with 2 topn, 3 buckets; +--sorted_result +show stats_topn where db_name = 'planner__cardinality__selectivity'; +--sorted_result +show stats_buckets where db_name = 'planner__cardinality__selectivity'; +explain select * from topn_before_hist where a = 1; +explain select * from topn_before_hist where a = 2; +explain select * from topn_after_hist where a = 7; +explain select * from topn_after_hist where a = 6; +explain select * from topn_after_hist_no_index where a = 7; +explain select * from topn_after_hist_no_index where a = 6; +explain select * from topn_before_hist_no_index where a = 1; +explain select * from topn_before_hist_no_index where a = 2; + +# TestDiscreteDistribution +drop table if exists t; +create table t(a char(10), b int, key idx(a, b)); +insert into t values ('cn', 0); +insert into t values ('cn', 1); +insert into t values ('cn', 2); +insert into t values ('cn', 3); +insert into t values ('cn', 4); +insert into t values ('cn', 5); +insert into t values ('cn', 6); +insert into t values ('cn', 7); +insert into t values ('cn', 8); +insert into t values ('cn', 9); +insert into t values ('cn', 10); +insert into t values ('cn', 11); +insert into t values ('cn', 12); +insert into t values ('cn', 13); +insert into t values ('cn', 14); +insert into t values ('cn', 15); +insert into t values ('cn', 16); +insert into t values ('cn', 17); +insert into t values ('cn', 18); +insert into t values ('cn', 19); +insert into t values ('cn', 20); +insert into t values ('cn', 21); +insert into t values ('cn', 22); +insert into t values ('cn', 23); +insert into t values ('cn', 24); +insert into t values ('cn', 25); +insert into t values ('cn', 26); +insert into t values ('cn', 27); +insert into t values ('cn', 28); +insert into t values ('cn', 29); +insert into t values ('cn', 30); +insert into t values ('cn', 31); +insert into t values ('cn', 32); +insert into t values ('cn', 33); +insert into t values ('cn', 34); +insert into t values ('cn', 35); +insert into t values ('cn', 36); +insert into t values ('cn', 37); +insert into t values ('cn', 38); +insert into t values ('cn', 39); +insert into t values ('cn', 40); +insert into t values ('cn', 41); +insert into t values ('cn', 42); +insert into t values ('cn', 43); +insert into t values ('cn', 44); +insert into t values ('cn', 45); +insert into t values ('cn', 46); +insert into t values ('cn', 47); +insert into t values ('cn', 48); +insert into t values ('cn', 49); +insert into t values ('cn', 50); +insert into t values ('cn', 51); +insert into t values ('cn', 52); +insert into t values ('cn', 53); +insert into t values ('cn', 54); +insert into t values ('cn', 55); +insert into t values ('cn', 56); +insert into t values ('cn', 57); +insert into t values ('cn', 58); +insert into t values ('cn', 59); +insert into t values ('cn', 60); +insert into t values ('cn', 61); +insert into t values ('cn', 62); +insert into t values ('cn', 63); +insert into t values ('cn', 64); +insert into t values ('cn', 65); +insert into t values ('cn', 66); +insert into t values ('cn', 67); +insert into t values ('cn', 68); +insert into t values ('cn', 69); +insert into t values ('cn', 70); +insert into t values ('cn', 71); +insert into t values ('cn', 72); +insert into t values ('cn', 73); +insert into t values ('cn', 74); +insert into t values ('cn', 75); +insert into t values ('cn', 76); +insert into t values ('cn', 77); +insert into t values ('cn', 78); +insert into t values ('cn', 79); +insert into t values ('cn', 80); +insert into t values ('cn', 81); +insert into t values ('cn', 82); +insert into t values ('cn', 83); +insert into t values ('cn', 84); +insert into t values ('cn', 85); +insert into t values ('cn', 86); +insert into t values ('cn', 87); +insert into t values ('cn', 88); +insert into t values ('cn', 89); +insert into t values ('cn', 90); +insert into t values ('cn', 91); +insert into t values ('cn', 92); +insert into t values ('cn', 93); +insert into t values ('cn', 94); +insert into t values ('cn', 95); +insert into t values ('cn', 96); +insert into t values ('cn', 97); +insert into t values ('cn', 98); +insert into t values ('cn', 99); +insert into t values ('cn', 100); +insert into t values ('cn', 101); +insert into t values ('cn', 102); +insert into t values ('cn', 103); +insert into t values ('cn', 104); +insert into t values ('cn', 105); +insert into t values ('cn', 106); +insert into t values ('cn', 107); +insert into t values ('cn', 108); +insert into t values ('cn', 109); +insert into t values ('cn', 110); +insert into t values ('cn', 111); +insert into t values ('cn', 112); +insert into t values ('cn', 113); +insert into t values ('cn', 114); +insert into t values ('cn', 115); +insert into t values ('cn', 116); +insert into t values ('cn', 117); +insert into t values ('cn', 118); +insert into t values ('cn', 119); +insert into t values ('cn', 120); +insert into t values ('cn', 121); +insert into t values ('cn', 122); +insert into t values ('cn', 123); +insert into t values ('cn', 124); +insert into t values ('cn', 125); +insert into t values ('cn', 126); +insert into t values ('cn', 127); +insert into t values ('cn', 128); +insert into t values ('cn', 129); +insert into t values ('cn', 130); +insert into t values ('cn', 131); +insert into t values ('cn', 132); +insert into t values ('cn', 133); +insert into t values ('cn', 134); +insert into t values ('cn', 135); +insert into t values ('cn', 136); +insert into t values ('cn', 137); +insert into t values ('cn', 138); +insert into t values ('cn', 139); +insert into t values ('cn', 140); +insert into t values ('cn', 141); +insert into t values ('cn', 142); +insert into t values ('cn', 143); +insert into t values ('cn', 144); +insert into t values ('cn', 145); +insert into t values ('cn', 146); +insert into t values ('cn', 147); +insert into t values ('cn', 148); +insert into t values ('cn', 149); +insert into t values ('cn', 150); +insert into t values ('cn', 151); +insert into t values ('cn', 152); +insert into t values ('cn', 153); +insert into t values ('cn', 154); +insert into t values ('cn', 155); +insert into t values ('cn', 156); +insert into t values ('cn', 157); +insert into t values ('cn', 158); +insert into t values ('cn', 159); +insert into t values ('cn', 160); +insert into t values ('cn', 161); +insert into t values ('cn', 162); +insert into t values ('cn', 163); +insert into t values ('cn', 164); +insert into t values ('cn', 165); +insert into t values ('cn', 166); +insert into t values ('cn', 167); +insert into t values ('cn', 168); +insert into t values ('cn', 169); +insert into t values ('cn', 170); +insert into t values ('cn', 171); +insert into t values ('cn', 172); +insert into t values ('cn', 173); +insert into t values ('cn', 174); +insert into t values ('cn', 175); +insert into t values ('cn', 176); +insert into t values ('cn', 177); +insert into t values ('cn', 178); +insert into t values ('cn', 179); +insert into t values ('cn', 180); +insert into t values ('cn', 181); +insert into t values ('cn', 182); +insert into t values ('cn', 183); +insert into t values ('cn', 184); +insert into t values ('cn', 185); +insert into t values ('cn', 186); +insert into t values ('cn', 187); +insert into t values ('cn', 188); +insert into t values ('cn', 189); +insert into t values ('cn', 190); +insert into t values ('cn', 191); +insert into t values ('cn', 192); +insert into t values ('cn', 193); +insert into t values ('cn', 194); +insert into t values ('cn', 195); +insert into t values ('cn', 196); +insert into t values ('cn', 197); +insert into t values ('cn', 198); +insert into t values ('cn', 199); +insert into t values ('cn', 200); +insert into t values ('cn', 201); +insert into t values ('cn', 202); +insert into t values ('cn', 203); +insert into t values ('cn', 204); +insert into t values ('cn', 205); +insert into t values ('cn', 206); +insert into t values ('cn', 207); +insert into t values ('cn', 208); +insert into t values ('cn', 209); +insert into t values ('cn', 210); +insert into t values ('cn', 211); +insert into t values ('cn', 212); +insert into t values ('cn', 213); +insert into t values ('cn', 214); +insert into t values ('cn', 215); +insert into t values ('cn', 216); +insert into t values ('cn', 217); +insert into t values ('cn', 218); +insert into t values ('cn', 219); +insert into t values ('cn', 220); +insert into t values ('cn', 221); +insert into t values ('cn', 222); +insert into t values ('cn', 223); +insert into t values ('cn', 224); +insert into t values ('cn', 225); +insert into t values ('cn', 226); +insert into t values ('cn', 227); +insert into t values ('cn', 228); +insert into t values ('cn', 229); +insert into t values ('cn', 230); +insert into t values ('cn', 231); +insert into t values ('cn', 232); +insert into t values ('cn', 233); +insert into t values ('cn', 234); +insert into t values ('cn', 235); +insert into t values ('cn', 236); +insert into t values ('cn', 237); +insert into t values ('cn', 238); +insert into t values ('cn', 239); +insert into t values ('cn', 240); +insert into t values ('cn', 241); +insert into t values ('cn', 242); +insert into t values ('cn', 243); +insert into t values ('cn', 244); +insert into t values ('cn', 245); +insert into t values ('cn', 246); +insert into t values ('cn', 247); +insert into t values ('cn', 248); +insert into t values ('cn', 249); +insert into t values ('cn', 250); +insert into t values ('cn', 251); +insert into t values ('cn', 252); +insert into t values ('cn', 253); +insert into t values ('cn', 254); +insert into t values ('cn', 255); +insert into t values ('cn', 256); +insert into t values ('cn', 257); +insert into t values ('cn', 258); +insert into t values ('cn', 259); +insert into t values ('cn', 260); +insert into t values ('cn', 261); +insert into t values ('cn', 262); +insert into t values ('cn', 263); +insert into t values ('cn', 264); +insert into t values ('cn', 265); +insert into t values ('cn', 266); +insert into t values ('cn', 267); +insert into t values ('cn', 268); +insert into t values ('cn', 269); +insert into t values ('cn', 270); +insert into t values ('cn', 271); +insert into t values ('cn', 272); +insert into t values ('cn', 273); +insert into t values ('cn', 274); +insert into t values ('cn', 275); +insert into t values ('cn', 276); +insert into t values ('cn', 277); +insert into t values ('cn', 278); +insert into t values ('cn', 279); +insert into t values ('cn', 280); +insert into t values ('cn', 281); +insert into t values ('cn', 282); +insert into t values ('cn', 283); +insert into t values ('cn', 284); +insert into t values ('cn', 285); +insert into t values ('cn', 286); +insert into t values ('cn', 287); +insert into t values ('cn', 288); +insert into t values ('cn', 289); +insert into t values ('cn', 290); +insert into t values ('cn', 291); +insert into t values ('cn', 292); +insert into t values ('cn', 293); +insert into t values ('cn', 294); +insert into t values ('cn', 295); +insert into t values ('cn', 296); +insert into t values ('cn', 297); +insert into t values ('cn', 298); +insert into t values ('cn', 299); +insert into t values ('cn', 300); +insert into t values ('cn', 301); +insert into t values ('cn', 302); +insert into t values ('cn', 303); +insert into t values ('cn', 304); +insert into t values ('cn', 305); +insert into t values ('cn', 306); +insert into t values ('cn', 307); +insert into t values ('cn', 308); +insert into t values ('cn', 309); +insert into t values ('cn', 310); +insert into t values ('cn', 311); +insert into t values ('cn', 312); +insert into t values ('cn', 313); +insert into t values ('cn', 314); +insert into t values ('cn', 315); +insert into t values ('cn', 316); +insert into t values ('cn', 317); +insert into t values ('cn', 318); +insert into t values ('cn', 319); +insert into t values ('cn', 320); +insert into t values ('cn', 321); +insert into t values ('cn', 322); +insert into t values ('cn', 323); +insert into t values ('cn', 324); +insert into t values ('cn', 325); +insert into t values ('cn', 326); +insert into t values ('cn', 327); +insert into t values ('cn', 328); +insert into t values ('cn', 329); +insert into t values ('cn', 330); +insert into t values ('cn', 331); +insert into t values ('cn', 332); +insert into t values ('cn', 333); +insert into t values ('cn', 334); +insert into t values ('cn', 335); +insert into t values ('cn', 336); +insert into t values ('cn', 337); +insert into t values ('cn', 338); +insert into t values ('cn', 339); +insert into t values ('cn', 340); +insert into t values ('cn', 341); +insert into t values ('cn', 342); +insert into t values ('cn', 343); +insert into t values ('cn', 344); +insert into t values ('cn', 345); +insert into t values ('cn', 346); +insert into t values ('cn', 347); +insert into t values ('cn', 348); +insert into t values ('cn', 349); +insert into t values ('cn', 350); +insert into t values ('cn', 351); +insert into t values ('cn', 352); +insert into t values ('cn', 353); +insert into t values ('cn', 354); +insert into t values ('cn', 355); +insert into t values ('cn', 356); +insert into t values ('cn', 357); +insert into t values ('cn', 358); +insert into t values ('cn', 359); +insert into t values ('cn', 360); +insert into t values ('cn', 361); +insert into t values ('cn', 362); +insert into t values ('cn', 363); +insert into t values ('cn', 364); +insert into t values ('cn', 365); +insert into t values ('cn', 366); +insert into t values ('cn', 367); +insert into t values ('cn', 368); +insert into t values ('cn', 369); +insert into t values ('cn', 370); +insert into t values ('cn', 371); +insert into t values ('cn', 372); +insert into t values ('cn', 373); +insert into t values ('cn', 374); +insert into t values ('cn', 375); +insert into t values ('cn', 376); +insert into t values ('cn', 377); +insert into t values ('cn', 378); +insert into t values ('cn', 379); +insert into t values ('cn', 380); +insert into t values ('cn', 381); +insert into t values ('cn', 382); +insert into t values ('cn', 383); +insert into t values ('cn', 384); +insert into t values ('cn', 385); +insert into t values ('cn', 386); +insert into t values ('cn', 387); +insert into t values ('cn', 388); +insert into t values ('cn', 389); +insert into t values ('cn', 390); +insert into t values ('cn', 391); +insert into t values ('cn', 392); +insert into t values ('cn', 393); +insert into t values ('cn', 394); +insert into t values ('cn', 395); +insert into t values ('cn', 396); +insert into t values ('cn', 397); +insert into t values ('cn', 398); +insert into t values ('cn', 399); +insert into t values ('cn', 400); +insert into t values ('cn', 401); +insert into t values ('cn', 402); +insert into t values ('cn', 403); +insert into t values ('cn', 404); +insert into t values ('cn', 405); +insert into t values ('cn', 406); +insert into t values ('cn', 407); +insert into t values ('cn', 408); +insert into t values ('cn', 409); +insert into t values ('cn', 410); +insert into t values ('cn', 411); +insert into t values ('cn', 412); +insert into t values ('cn', 413); +insert into t values ('cn', 414); +insert into t values ('cn', 415); +insert into t values ('cn', 416); +insert into t values ('cn', 417); +insert into t values ('cn', 418); +insert into t values ('cn', 419); +insert into t values ('cn', 420); +insert into t values ('cn', 421); +insert into t values ('cn', 422); +insert into t values ('cn', 423); +insert into t values ('cn', 424); +insert into t values ('cn', 425); +insert into t values ('cn', 426); +insert into t values ('cn', 427); +insert into t values ('cn', 428); +insert into t values ('cn', 429); +insert into t values ('cn', 430); +insert into t values ('cn', 431); +insert into t values ('cn', 432); +insert into t values ('cn', 433); +insert into t values ('cn', 434); +insert into t values ('cn', 435); +insert into t values ('cn', 436); +insert into t values ('cn', 437); +insert into t values ('cn', 438); +insert into t values ('cn', 439); +insert into t values ('cn', 440); +insert into t values ('cn', 441); +insert into t values ('cn', 442); +insert into t values ('cn', 443); +insert into t values ('cn', 444); +insert into t values ('cn', 445); +insert into t values ('cn', 446); +insert into t values ('cn', 447); +insert into t values ('cn', 448); +insert into t values ('cn', 449); +insert into t values ('cn', 450); +insert into t values ('cn', 451); +insert into t values ('cn', 452); +insert into t values ('cn', 453); +insert into t values ('cn', 454); +insert into t values ('cn', 455); +insert into t values ('cn', 456); +insert into t values ('cn', 457); +insert into t values ('cn', 458); +insert into t values ('cn', 459); +insert into t values ('cn', 460); +insert into t values ('cn', 461); +insert into t values ('cn', 462); +insert into t values ('cn', 463); +insert into t values ('cn', 464); +insert into t values ('cn', 465); +insert into t values ('cn', 466); +insert into t values ('cn', 467); +insert into t values ('cn', 468); +insert into t values ('cn', 469); +insert into t values ('cn', 470); +insert into t values ('cn', 471); +insert into t values ('cn', 472); +insert into t values ('cn', 473); +insert into t values ('cn', 474); +insert into t values ('cn', 475); +insert into t values ('cn', 476); +insert into t values ('cn', 477); +insert into t values ('cn', 478); +insert into t values ('cn', 479); +insert into t values ('cn', 480); +insert into t values ('cn', 481); +insert into t values ('cn', 482); +insert into t values ('cn', 483); +insert into t values ('cn', 484); +insert into t values ('cn', 485); +insert into t values ('cn', 486); +insert into t values ('cn', 487); +insert into t values ('cn', 488); +insert into t values ('cn', 489); +insert into t values ('cn', 490); +insert into t values ('cn', 491); +insert into t values ('cn', 492); +insert into t values ('cn', 493); +insert into t values ('cn', 494); +insert into t values ('cn', 495); +insert into t values ('cn', 496); +insert into t values ('cn', 497); +insert into t values ('cn', 498); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +insert into t values ('tw', 0); +analyze table t all columns; +explain select * from t where a = 'tw' and b < 0; +set @@tidb_opt_fix_control = '47400:on'; +explain select * from t where a = 'tw' and b < 0; +set @@tidb_opt_fix_control = '47400:off'; + +# TestSelectCombinedLowBound +drop table if exists t; +create table t(id int auto_increment, kid int, pid int, primary key(id), key(kid, pid)); +insert into t (kid, pid) values (1,2), (1,3), (1,4),(1, 11), (1, 12), (1, 13), (1, 14), (2, 2), (2, 3), (2, 4); +analyze table t all columns; +explain select * from t where kid = 1; + +# TestDefaultSelectivityForStrMatch +drop table if exists t; +create table t(a int, b varchar(100)); +set @@tidb_default_string_match_selectivity = 0.8; +explain format = 'brief' select * from t where a + 10 > 100; +explain format = 'brief' select * from t where b like '%test%'; +explain format = 'brief' select * from t where b not like '%test%'; +explain format = 'brief' select * from t where b regexp '.*test.*'; +explain format = 'brief' select * from t where b not regexp '.*test.*'; +explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100; +explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100; +explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100; +set @@tidb_default_string_match_selectivity = 0.9; +explain format = 'brief' select * from t where a + 10 > 100; +explain format = 'brief' select * from t where b like '%test%'; +explain format = 'brief' select * from t where b not like '%test%'; +explain format = 'brief' select * from t where b regexp '.*test.*'; +explain format = 'brief' select * from t where b not regexp '.*test.*'; +explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100; +explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100; +explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100; +set @@tidb_default_string_match_selectivity = 0.1; +explain format = 'brief' select * from t where a + 10 > 100; +explain format = 'brief' select * from t where b like '%test%'; +explain format = 'brief' select * from t where b like '%test%' is true; +explain format = 'brief' select * from t where b not like '%test%'; +explain format = 'brief' select * from t where b regexp '.*test.*'; +explain format = 'brief' select * from t where b rlike '.*test.*'; +explain format = 'brief' select * from t where b not rlike '.*test.*'; +explain format = 'brief' select * from t where b not regexp '.*test.*'; +explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100; +explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100; +explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100; +set @@tidb_default_string_match_selectivity = 0; +explain format = 'brief' select * from t where a + 10 > 100; +explain format = 'brief' select * from t where b like '%test%'; +explain format = 'brief' select * from t where b not like '%test%'; +explain format = 'brief' select * from t where b regexp '.*test.*'; +explain format = 'brief' select * from t where b not regexp '.*test.*'; +explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100; +explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100; +explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100; +