diff --git a/build/nogo_config.json b/build/nogo_config.json index 97c68b6f486a5..a3e00bea7e142 100644 --- a/build/nogo_config.json +++ b/build/nogo_config.json @@ -338,6 +338,7 @@ }, "lll": { "exclude_files": { + ".*_test\\.go$": "ignore test code", "/cgo/": "ignore cgo code", "external/": "no need to vet third party code", ".*_generated\\.go$": "ignore generated code", diff --git a/planner/cardinality/BUILD.bazel b/planner/cardinality/BUILD.bazel new file mode 100644 index 0000000000000..df150f154612d --- /dev/null +++ b/planner/cardinality/BUILD.bazel @@ -0,0 +1,70 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "cardinality", + srcs = [ + "pseudo.go", + "selectivity.go", + "trace.go", + ], + importpath = "github.com/pingcap/tidb/planner/cardinality", + visibility = ["//visibility:public"], + deps = [ + "//expression", + "//parser/ast", + "//parser/format", + "//parser/mysql", + "//planner/util", + "//planner/util/debugtrace", + "//sessionctx", + "//statistics", + "//types", + "//types/parser_driver", + "//util/chunk", + "//util/logutil", + "//util/ranger", + "//util/tracing", + "@com_github_pingcap_errors//:errors", + "@org_golang_x_exp//maps", + "@org_uber_go_zap//:zap", + ], +) + +go_test( + name = "cardinality_test", + timeout = "short", + srcs = [ + "main_test.go", + "selectivity_test.go", + ], + data = glob(["testdata/**"]), + embed = [":cardinality"], + flaky = True, + shard_count = 25, + deps = [ + "//config", + "//domain", + "//kv", + "//parser/model", + "//parser/mysql", + "//planner/core", + "//session", + "//sessionctx", + "//sessionctx/stmtctx", + "//sessionctx/variable", + "//statistics", + "//statistics/handle", + "//testkit", + "//testkit/testdata", + "//testkit/testmain", + "//testkit/testsetup", + "//types", + "//util/codec", + "//util/collate", + "//util/mock", + "//util/ranger", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/planner/cardinality/main_test.go b/planner/cardinality/main_test.go new file mode 100644 index 0000000000000..d4c80f050638e --- /dev/null +++ b/planner/cardinality/main_test.go @@ -0,0 +1,65 @@ +// 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 ( + "flag" + "testing" + + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/testkit/testmain" + "github.com/pingcap/tidb/testkit/testsetup" + "go.uber.org/goleak" +) + +var testDataMap = make(testdata.BookKeeper, 3) + +func TestMain(m *testing.M) { + testsetup.SetupForCommonTest() + + if !flag.Parsed() { + flag.Parse() + } + + config.UpdateGlobal(func(conf *config.Config) { + conf.TiKVClient.AsyncCommit.SafeWindow = 0 + conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 + }) + + testDataMap.LoadTestSuiteData("testdata", "cardinality_suite") + + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + + callback := func(i int) int { + testDataMap.GenerateOutputIfNeeded() + return i + } + goleak.VerifyTestMain(testmain.WrapTestingM(m, callback), opts...) +} + +func GetCardinalitySuiteData() testdata.TestData { + return testDataMap["cardinality_suite"] +} + +// MockStatsNode is only used for test. +func MockStatsNode(id int64, m int64, num int) *StatsNode { + return &StatsNode{ID: id, mask: m, numCols: num} +} diff --git a/planner/cardinality/pseudo.go b/planner/cardinality/pseudo.go new file mode 100644 index 0000000000000..98e0095353a8b --- /dev/null +++ b/planner/cardinality/pseudo.go @@ -0,0 +1,82 @@ +// 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 ( + "math" + + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/statistics" +) + +const ( + pseudoEqualRate = 1000 + pseudoLessRate = 3 +) + +// If one condition can't be calculated, we will assume that the selectivity of this condition is 0.8. +const selectionFactor = 0.8 + +func pseudoSelectivity(coll *statistics.HistColl, exprs []expression.Expression) float64 { + minFactor := selectionFactor + colExists := make(map[string]bool) + for _, expr := range exprs { + fun, ok := expr.(*expression.ScalarFunction) + if !ok { + continue + } + colID := getConstantColumnID(fun.GetArgs()) + if colID == unknownColumnID { + continue + } + switch fun.FuncName.L { + case ast.EQ, ast.NullEQ, ast.In: + minFactor = math.Min(minFactor, 1.0/pseudoEqualRate) + col, ok := coll.Columns[colID] + if !ok { + continue + } + colExists[col.Info.Name.L] = true + if mysql.HasUniKeyFlag(col.Info.GetFlag()) { + return 1.0 / float64(coll.RealtimeCount) + } + case ast.GE, ast.GT, ast.LE, ast.LT: + minFactor = math.Min(minFactor, 1.0/pseudoLessRate) + // FIXME: To resolve the between case. + } + } + if len(colExists) == 0 { + return minFactor + } + // use the unique key info + for _, idx := range coll.Indices { + if !idx.Info.Unique { + continue + } + unique := true + for _, col := range idx.Info.Columns { + if !colExists[col.Name.L] { + unique = false + break + } + } + if unique { + return 1.0 / float64(coll.RealtimeCount) + } + } + return minFactor +} diff --git a/planner/cardinality/selectivity.go b/planner/cardinality/selectivity.go new file mode 100644 index 0000000000000..fcb8ab7f0a79b --- /dev/null +++ b/planner/cardinality/selectivity.go @@ -0,0 +1,615 @@ +// 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 ( + "cmp" + "math" + "math/bits" + "slices" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/parser/ast" + planutil "github.com/pingcap/tidb/planner/util" + "github.com/pingcap/tidb/planner/util/debugtrace" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/ranger" + "go.uber.org/zap" + "golang.org/x/exp/maps" +) + +// Selectivity is a function calculate the selectivity of the expressions on the specified HistColl. +// The definition of selectivity is (row count after filter / row count before filter). +// And exprs must be CNF now, in other words, `exprs[0] and exprs[1] and ... and exprs[len - 1]` +// should be held when you call this. +// Currently the time complexity is o(n^2). +func Selectivity( + ctx sessionctx.Context, + coll *statistics.HistColl, + exprs []expression.Expression, + filledPaths []*planutil.AccessPath, +) ( + result float64, + retStatsNodes []*StatsNode, + err error, +) { + var exprStrs []string + if ctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(ctx) + exprStrs = expression.ExprsToStringsForDisplay(exprs) + debugtrace.RecordAnyValuesWithNames(ctx, "Input Expressions", exprStrs) + defer func() { + debugtrace.RecordAnyValuesWithNames(ctx, "Result", result) + debugtrace.LeaveContextCommon(ctx) + }() + } + // If table's count is zero or conditions are empty, we should return 100% selectivity. + if coll.RealtimeCount == 0 || len(exprs) == 0 { + return 1, nil, nil + } + ret := 1.0 + sc := ctx.GetSessionVars().StmtCtx + tableID := coll.PhysicalID + // TODO: If len(exprs) is bigger than 63, we could use bitset structure to replace the int64. + // This will simplify some code and speed up if we use this rather than a boolean slice. + if len(exprs) > 63 || (len(coll.Columns) == 0 && len(coll.Indices) == 0) { + ret = pseudoSelectivity(coll, exprs) + if sc.EnableOptimizerCETrace { + CETraceExpr(ctx, tableID, "Table Stats-Pseudo-Expression", + expression.ComposeCNFCondition(ctx, exprs...), ret*float64(coll.RealtimeCount)) + } + return ret, nil, nil + } + + var nodes []*StatsNode + + var remainedExprStrs []string + remainedExprs := make([]expression.Expression, 0, len(exprs)) + + // Deal with the correlated column. + for i, expr := range exprs { + c := isColEqCorCol(expr) + if c == nil { + remainedExprs = append(remainedExprs, expr) + if sc.EnableOptimizerDebugTrace { + remainedExprStrs = append(remainedExprStrs, exprStrs[i]) + } + continue + } + + colHist := coll.Columns[c.UniqueID] + var sel float64 + if colHist == nil || colHist.IsInvalid(ctx, coll.Pseudo) { + sel = 1.0 / pseudoEqualRate + } else if colHist.Histogram.NDV > 0 { + sel = 1 / float64(colHist.Histogram.NDV) + } else { + sel = 1.0 / pseudoEqualRate + } + if sc.EnableOptimizerDebugTrace { + debugtrace.RecordAnyValuesWithNames(ctx, "Expression", expr.String(), "Selectivity", sel) + } + ret *= sel + } + + extractedCols := make([]*expression.Column, 0, len(coll.Columns)) + extractedCols = expression.ExtractColumnsFromExpressions(extractedCols, remainedExprs, nil) + colIDs := maps.Keys(coll.Columns) + slices.Sort(colIDs) + for _, id := range colIDs { + colStats := coll.Columns[id] + col := expression.ColInfo2Col(extractedCols, colStats.Info) + if col != nil { + maskCovered, ranges, _, err := getMaskAndRanges(ctx, remainedExprs, ranger.ColumnRangeType, nil, nil, col) + if err != nil { + return 0, nil, errors.Trace(err) + } + nodes = append(nodes, &StatsNode{Tp: ColType, ID: id, mask: maskCovered, Ranges: ranges, numCols: 1}) + if colStats.IsHandle { + nodes[len(nodes)-1].Tp = PkType + var cnt float64 + cnt, err = coll.GetRowCountByIntColumnRanges(ctx, id, ranges) + if err != nil { + return 0, nil, errors.Trace(err) + } + nodes[len(nodes)-1].Selectivity = cnt / float64(coll.RealtimeCount) + continue + } + cnt, err := coll.GetRowCountByColumnRanges(ctx, id, ranges) + if err != nil { + return 0, nil, errors.Trace(err) + } + nodes[len(nodes)-1].Selectivity = cnt / float64(coll.RealtimeCount) + } + } + id2Paths := make(map[int64]*planutil.AccessPath) + for _, path := range filledPaths { + // Index merge path and table path don't have index. + if path.Index == nil { + continue + } + id2Paths[path.Index.ID] = path + } + idxIDs := maps.Keys(coll.Indices) + slices.Sort(idxIDs) + for _, id := range idxIDs { + idxStats := coll.Indices[id] + idxCols := FindPrefixOfIndexByCol(extractedCols, coll.Idx2ColumnIDs[id], id2Paths[idxStats.ID]) + if len(idxCols) > 0 { + lengths := make([]int, 0, len(idxCols)) + for i := 0; i < len(idxCols) && i < len(idxStats.Info.Columns); i++ { + lengths = append(lengths, idxStats.Info.Columns[i].Length) + } + // If the found columns are more than the columns held by the index. We are appending the int pk to the tail of it. + // When storing index data to key-value store, we use (idx_col1, ...., idx_coln, handle_col) as its key. + if len(idxCols) > len(idxStats.Info.Columns) { + lengths = append(lengths, types.UnspecifiedLength) + } + maskCovered, ranges, partCover, err := getMaskAndRanges(ctx, remainedExprs, + ranger.IndexRangeType, lengths, id2Paths[idxStats.ID], idxCols...) + if err != nil { + return 0, nil, errors.Trace(err) + } + cnt, err := coll.GetRowCountByIndexRanges(ctx, id, ranges) + if err != nil { + return 0, nil, errors.Trace(err) + } + selectivity := cnt / float64(coll.RealtimeCount) + nodes = append(nodes, &StatsNode{ + Tp: IndexType, + ID: id, + mask: maskCovered, + Ranges: ranges, + numCols: len(idxStats.Info.Columns), + Selectivity: selectivity, + partCover: partCover, + }) + } + } + usedSets := GetUsableSetsByGreedy(nodes) + // Initialize the mask with the full set. + mask := (int64(1) << uint(len(remainedExprs))) - 1 + // curExpr records covered expressions by now. It's for cardinality estimation tracing. + var curExpr []expression.Expression + + for _, set := range usedSets { + mask &^= set.mask + ret *= set.Selectivity + // If `partCover` is true, it means that the conditions are in DNF form, and only part + // of the DNF expressions are extracted as access conditions, so besides from the selectivity + // of the extracted access conditions, we multiply another selectionFactor for the residual + // conditions. + if set.partCover { + ret *= selectionFactor + } + if sc.EnableOptimizerCETrace { + // Tracing for the expression estimation results after applying this StatsNode. + for i := range remainedExprs { + if set.mask&(1< 0 { + curExpr = append(curExpr, remainedExprs[i]) + } + } + expr := expression.ComposeCNFCondition(ctx, curExpr...) + CETraceExpr(ctx, tableID, "Table Stats-Expression-CNF", expr, ret*float64(coll.RealtimeCount)) + } else if sc.EnableOptimizerDebugTrace { + var strs []string + for i := range remainedExprs { + if set.mask&(1< 0 { + strs = append(strs, remainedExprStrs[i]) + } + } + debugtrace.RecordAnyValuesWithNames(ctx, + "Expressions", strs, + "Selectivity", set.Selectivity, + "partial cover", set.partCover, + ) + } + } + + notCoveredConstants := make(map[int]*expression.Constant) + notCoveredDNF := make(map[int]*expression.ScalarFunction) + notCoveredStrMatch := make(map[int]*expression.ScalarFunction) + notCoveredNegateStrMatch := make(map[int]*expression.ScalarFunction) + notCoveredOtherExpr := make(map[int]expression.Expression) + if mask > 0 { + for i, expr := range remainedExprs { + if mask&(1< 0 { + minSelectivity := 1.0 + if len(notCoveredConstants) > 0 || len(notCoveredDNF) > 0 || len(notCoveredOtherExpr) > 0 { + minSelectivity = math.Min(minSelectivity, selectionFactor) + } + if len(notCoveredStrMatch) > 0 { + minSelectivity = math.Min(minSelectivity, ctx.GetSessionVars().GetStrMatchDefaultSelectivity()) + } + if len(notCoveredNegateStrMatch) > 0 { + minSelectivity = math.Min(minSelectivity, ctx.GetSessionVars().GetNegateStrMatchDefaultSelectivity()) + } + ret *= minSelectivity + if sc.EnableOptimizerDebugTrace { + debugtrace.RecordAnyValuesWithNames(ctx, "Default Selectivity", minSelectivity) + } + } + + if sc.EnableOptimizerCETrace { + // Tracing for the expression estimation results after applying the default selectivity. + totalExpr := expression.ComposeCNFCondition(ctx, remainedExprs...) + CETraceExpr(ctx, tableID, "Table Stats-Expression-CNF", totalExpr, ret*float64(coll.RealtimeCount)) + } + return ret, nodes, nil +} + +// StatsNode is used for calculating selectivity. +type StatsNode struct { + // Ranges contains all the Ranges we got. + Ranges []*ranger.Range + Tp int + ID int64 + // mask is a bit pattern whose ith bit will indicate whether the ith expression is covered by this index/column. + mask int64 + // Selectivity indicates the Selectivity of this column/index. + Selectivity float64 + // numCols is the number of columns contained in the index or column(which is always 1). + numCols int + // partCover indicates whether the bit in the mask is for a full cover or partial cover. It is only true + // when the condition is a DNF expression on index, and the expression is not totally extracted as access condition. + partCover bool +} + +// The type of the StatsNode. +const ( + IndexType = iota + PkType + ColType +) + +func compareType(l, r int) int { + if l == r { + return 0 + } + if l == ColType { + return -1 + } + if l == PkType { + return 1 + } + if r == ColType { + return 1 + } + return -1 +} + +const unknownColumnID = math.MinInt64 + +// getConstantColumnID receives two expressions and if one of them is column and another is constant, it returns the +// ID of the column. +func getConstantColumnID(e []expression.Expression) int64 { + if len(e) != 2 { + return unknownColumnID + } + col, ok1 := e[0].(*expression.Column) + _, ok2 := e[1].(*expression.Constant) + if ok1 && ok2 { + return col.ID + } + col, ok1 = e[1].(*expression.Column) + _, ok2 = e[0].(*expression.Constant) + if ok1 && ok2 { + return col.ID + } + return unknownColumnID +} + +// GetUsableSetsByGreedy will select the indices and pk used for calculate selectivity by greedy algorithm. +func GetUsableSetsByGreedy(nodes []*StatsNode) (newBlocks []*StatsNode) { + slices.SortFunc(nodes, func(i, j *StatsNode) int { + if r := compareType(i.Tp, j.Tp); r != 0 { + return r + } + return cmp.Compare(i.ID, j.ID) + }) + marked := make([]bool, len(nodes)) + mask := int64(math.MaxInt64) + for { + // Choose the index that covers most. + bestID, bestCount, bestTp, bestNumCols, bestMask, bestSel := -1, 0, ColType, 0, int64(0), float64(0) + for i, set := range nodes { + if marked[i] { + continue + } + curMask := set.mask & mask + if curMask != set.mask { + marked[i] = true + continue + } + bits := bits.OnesCount64(uint64(curMask)) + // This set cannot cover any thing, just skip it. + if bits == 0 { + marked[i] = true + continue + } + // We greedy select the stats info based on: + // (1): The stats type, always prefer the primary key or index. + // (2): The number of expression that it covers, the more the better. + // (3): The number of columns that it contains, the less the better. + // (4): The selectivity of the covered conditions, the less the better. + // The rationale behind is that lower selectivity tends to reflect more functional dependencies + // between columns. It's hard to decide the priority of this rule against rule 2 and 3, in order + // to avoid massive plan changes between tidb-server versions, I adopt this conservative strategy + // to impose this rule after rule 2 and 3. + if (bestTp == ColType && set.Tp != ColType) || + bestCount < bits || + (bestCount == bits && bestNumCols > set.numCols) || + (bestCount == bits && bestNumCols == set.numCols && bestSel > set.Selectivity) { + bestID, bestCount, bestTp, bestNumCols, bestMask, bestSel = i, bits, set.Tp, set.numCols, curMask, set.Selectivity + } + } + if bestCount == 0 { + break + } + + // Update the mask, remove the bit that nodes[bestID].mask has. + mask &^= bestMask + + newBlocks = append(newBlocks, nodes[bestID]) + marked[bestID] = true + } + return +} + +// isColEqCorCol checks if the expression is a eq function that one side is correlated column and another is column. +// If so, it will return the column's reference. Otherwise return nil instead. +func isColEqCorCol(filter expression.Expression) *expression.Column { + f, ok := filter.(*expression.ScalarFunction) + if !ok || f.FuncName.L != ast.EQ { + return nil + } + if c, ok := f.GetArgs()[0].(*expression.Column); ok { + if _, ok := f.GetArgs()[1].(*expression.CorrelatedColumn); ok { + return c + } + } + if c, ok := f.GetArgs()[1].(*expression.Column); ok { + if _, ok := f.GetArgs()[0].(*expression.CorrelatedColumn); ok { + return c + } + } + return nil +} + +// FindPrefixOfIndexByCol will find columns in index by checking the unique id or the virtual expression. +// So it will return at once no matching column is found. +func FindPrefixOfIndexByCol(cols []*expression.Column, idxColIDs []int64, + cachedPath *planutil.AccessPath) []*expression.Column { + if cachedPath != nil { + idxCols := cachedPath.IdxCols + retCols := make([]*expression.Column, 0, len(idxCols)) + idLoop: + for _, idCol := range idxCols { + for _, col := range cols { + if col.EqualByExprAndID(nil, idCol) { + retCols = append(retCols, col) + continue idLoop + } + } + // If no matching column is found, just return. + return retCols + } + return retCols + } + return expression.FindPrefixOfIndex(cols, idxColIDs) +} + +func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, rangeType ranger.RangeType, + lengths []int, cachedPath *planutil.AccessPath, cols ...*expression.Column) ( + mask int64, ranges []*ranger.Range, partCover bool, err error) { + isDNF := false + var accessConds, remainedConds []expression.Expression + switch rangeType { + case ranger.ColumnRangeType: + accessConds = ranger.ExtractAccessConditionsForColumn(ctx, exprs, cols[0]) + ranges, accessConds, _, err = ranger.BuildColumnRange(accessConds, ctx, cols[0].RetType, + types.UnspecifiedLength, ctx.GetSessionVars().RangeMaxSize) + case ranger.IndexRangeType: + if cachedPath != nil { + ranges, accessConds, remainedConds, isDNF = cachedPath.Ranges, + cachedPath.AccessConds, cachedPath.TableFilters, cachedPath.IsDNFCond + break + } + var res *ranger.DetachRangeResult + res, err = ranger.DetachCondAndBuildRangeForIndex(ctx, exprs, cols, lengths, ctx.GetSessionVars().RangeMaxSize) + if err != nil { + return 0, nil, false, err + } + ranges, accessConds, remainedConds, isDNF = res.Ranges, res.AccessConds, res.RemainedConds, res.IsDNFCond + default: + panic("should never be here") + } + if err != nil { + return 0, nil, false, err + } + if isDNF && len(accessConds) > 0 { + mask |= 1 + return mask, ranges, len(remainedConds) > 0, nil + } + for i := range exprs { + for j := range accessConds { + if exprs[i].Equal(ctx, accessConds[j]) { + mask |= 1 << uint64(i) + break + } + } + } + return mask, ranges, false, nil +} diff --git a/statistics/selectivity_test.go b/planner/cardinality/selectivity_test.go similarity index 84% rename from statistics/selectivity_test.go rename to planner/cardinality/selectivity_test.go index cb063d1be3fd6..b18e20b9ec530 100644 --- a/statistics/selectivity_test.go +++ b/planner/cardinality/selectivity_test.go @@ -1,4 +1,4 @@ -// Copyright 2021 PingCAP, Inc. +// Copyright 2023 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package statistics_test +package cardinality_test import ( "context" @@ -22,6 +22,7 @@ import ( "regexp" "runtime/pprof" "slices" + "strings" "testing" "time" @@ -30,6 +31,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/cardinality" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" @@ -64,7 +66,7 @@ func TestCollationColumnEstimate(t *testing.T) { input []string output [][]string ) - statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i := 0; i < len(input); i++ { testdata.OnRecord(func() { @@ -103,7 +105,7 @@ func BenchmarkSelectivity(b *testing.B) { b.Run("Selectivity", func(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - _, _, err := statsTbl.Selectivity(sctx, p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection).Conditions, nil) + _, _, err := cardinality.Selectivity(sctx, &statsTbl.HistColl, p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection).Conditions, nil) require.NoError(b, err) } b.ReportAllocs() @@ -144,7 +146,7 @@ func TestOutOfRangeEstimation(t *testing.T) { End int64 Count float64 } - statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) increasedTblRowCount := int64(float64(statsTbl.RealtimeCount) * 1.5) modifyCount := int64(float64(statsTbl.RealtimeCount) * 0.5) @@ -186,7 +188,7 @@ func TestOutOfRangeEstimationAfterDelete(t *testing.T) { Result []string } ) - statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i := range input { testdata.OnRecord(func() { @@ -314,7 +316,7 @@ func TestPrimaryKeySelectivity(t *testing.T) { testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly testKit.MustExec("create table t(a char(10) primary key, b int)") var input, output [][]string - statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i, ts := range input { for j, tt := range ts { @@ -391,7 +393,7 @@ func TestStatsVer2(t *testing.T) { input []string output [][]string ) - statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i := range input { testdata.OnRecord(func() { @@ -428,7 +430,7 @@ func TestTopNOutOfHist(t *testing.T) { input []string output [][]string ) - statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i := range input { testdata.OnRecord(func() { @@ -452,7 +454,7 @@ func TestColumnIndexNullEstimation(t *testing.T) { input []string output [][]string ) - statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i := 0; i < 5; i++ { testdata.OnRecord(func() { @@ -486,7 +488,7 @@ func TestUniqCompEqualEst(t *testing.T) { input []string output [][]string ) - statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i := 0; i < 1; i++ { testdata.OnRecord(func() { @@ -576,13 +578,13 @@ func TestSelectivity(t *testing.T) { histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.TableInfo(), ds.Schema().Columns) - ratio, _, err := histColl.Selectivity(sctx, sel.Conditions, nil) + ratio, _, err := cardinality.Selectivity(sctx, histColl, sel.Conditions, nil) require.NoErrorf(t, err, "for %s", tt.exprs) require.Truef(t, math.Abs(ratio-tt.selectivity) < eps, "for %s, needed: %v, got: %v", tt.exprs, tt.selectivity, ratio) histColl.RealtimeCount *= 10 histColl.ModifyCount = histColl.RealtimeCount * 9 - ratio, _, err = histColl.Selectivity(sctx, sel.Conditions, nil) + ratio, _, err = cardinality.Selectivity(sctx, histColl, sel.Conditions, nil) require.NoErrorf(t, err, "for %s", tt.exprs) require.Truef(t, math.Abs(ratio-tt.selectivityAfterIncrease) < eps, "for %s, needed: %v, got: %v", tt.exprs, tt.selectivityAfterIncrease, ratio) } @@ -608,7 +610,7 @@ func TestDiscreteDistribution(t *testing.T) { output [][]string ) - statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i, tt := range input { @@ -632,7 +634,7 @@ func TestSelectCombinedLowBound(t *testing.T) { output [][]string ) - statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i, tt := range input { @@ -670,7 +672,7 @@ func TestDNFCondSelectivity(t *testing.T) { Selectivity float64 } ) - statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i, tt := range input { sctx := testKit.Session().(sessionctx.Context) @@ -689,7 +691,7 @@ func TestDNFCondSelectivity(t *testing.T) { histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.TableInfo(), ds.Schema().Columns) - ratio, _, err := histColl.Selectivity(sctx, sel.Conditions, nil) + ratio, _, err := cardinality.Selectivity(sctx, histColl, sel.Conditions, nil) require.NoErrorf(t, err, "error %v, for expr %s", err, tt) testdata.OnRecord(func() { output[i].SQL = tt @@ -703,7 +705,7 @@ func TestDNFCondSelectivity(t *testing.T) { // Test issue 22134 // Information about column n will not be in stats immediately after this SQL executed. - // If we don't have a check against this, DNF condition could lead to infinite recursion in Selectivity(). + // If we don't have a check against this, DNF condition could lead to infinite recursion in cardinality.Selectivity(). testKit.MustExec("alter table t add column n timestamp;") testKit.MustExec("select * from t where n = '2000-01-01' or n = '2000-01-02';") @@ -784,7 +786,7 @@ func TestSmallRangeEstimation(t *testing.T) { End int64 Count float64 } - statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i, ran := range input { count, err := col.GetColumnRowCount(sctx, getRange(ran.Start, ran.End), statsTbl.RealtimeCount, statsTbl.ModifyCount, false) @@ -917,19 +919,19 @@ func getRanges(start, end []int64) (res ranger.Ranges) { } func TestSelectivityGreedyAlgo(t *testing.T) { - nodes := make([]*statistics.StatsNode, 3) - nodes[0] = statistics.MockStatsNode(1, 3, 2) - nodes[1] = statistics.MockStatsNode(2, 5, 2) - nodes[2] = statistics.MockStatsNode(3, 9, 2) + nodes := make([]*cardinality.StatsNode, 3) + nodes[0] = cardinality.MockStatsNode(1, 3, 2) + nodes[1] = cardinality.MockStatsNode(2, 5, 2) + nodes[2] = cardinality.MockStatsNode(3, 9, 2) // Sets should not overlap on mask, so only nodes[0] is chosen. - usedSets := statistics.GetUsableSetsByGreedy(nodes) + usedSets := cardinality.GetUsableSetsByGreedy(nodes) require.Equal(t, 1, len(usedSets)) require.Equal(t, int64(1), usedSets[0].ID) nodes[0], nodes[1] = nodes[1], nodes[0] // Sets chosen should be stable, so the returned node is still the one with ID 1. - usedSets = statistics.GetUsableSetsByGreedy(nodes) + usedSets = cardinality.GetUsableSetsByGreedy(nodes) require.Equal(t, 1, len(usedSets)) require.Equal(t, int64(1), usedSets[0].ID) } @@ -949,7 +951,7 @@ func TestDefaultSelectivityForStrMatch(t *testing.T) { } ) - statsSuiteData := statistics.GetIntegrationSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) matchExplain, err := regexp.Compile("^explain") @@ -977,7 +979,7 @@ func TestTopNAssistedEstimationWithoutNewCollation(t *testing.T) { input []string output []outputType ) - statsSuiteData := statistics.GetIntegrationSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) testTopNAssistedEstimationInner(t, input, output, store, dom) } @@ -989,7 +991,7 @@ func TestTopNAssistedEstimationWithNewCollation(t *testing.T) { input []string output []outputType ) - statsSuiteData := statistics.GetIntegrationSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) testTopNAssistedEstimationInner(t, input, output, store, dom) } @@ -1076,7 +1078,7 @@ func TestGlobalStatsOutOfRangeEstimationAfterDelete(t *testing.T) { Result []string } ) - statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData := cardinality.GetCardinalitySuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i := range input { testdata.OnRecord(func() { @@ -1157,3 +1159,147 @@ func TestIssue39593(t *testing.T) { // 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(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.Columns[int64(i)] = &statistics.Column{ + Histogram: *mockStatsHistogram(int64(i), colValues, 1000, types.NewFieldType(mysql.TypeLonglong)), + Info: tblInfo.Columns[i-1], + StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), + StatsVer: 2, + } + } + generateMapsForMockStatsTbl(mockStatsTbl) + stat := h.GetTableStats(tblInfo) + stat.HistColl = mockStatsTbl.HistColl + + 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" + + testKit.MustQuery(query).Check(testkit.Rows( + "IndexJoin 1000000.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) 1000000.00 root ", + " ├─Selection(Build) 500000000.00 cop[tikv] 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) 1000000.00 cop[tikv] eq(test.t.a, 0)", + " └─TableRowIDScan 500000000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + )) + + testKit.MustExec("set @@tidb_opt_fix_control = '44855:ON'") + testKit.MustQuery(query).Check(testkit.Rows( + "IndexJoin 1000000.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) 1000000.00 root ", + " ├─Selection(Build) 1000000.00 cop[tikv] 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) 1000000.00 cop[tikv] eq(test.t.a, 0)", + " └─TableRowIDScan 1000000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + )) +} + +func TestOrderingIdxSelectivityThreshold(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + sc := &stmtctx.StatementContext{TimeZone: 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(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.Columns[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, nil, val) + require.NoError(t, err) + idxValues = append(idxValues, types.NewBytesDatum(b)) + } + + for i := 2; i <= 3; i++ { + mockStatsTbl.Columns[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.Indices[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...)) + } +} diff --git a/planner/cardinality/testdata/cardinality_suite_in.json b/planner/cardinality/testdata/cardinality_suite_in.json new file mode 100644 index 0000000000000..269169b6c648e --- /dev/null +++ b/planner/cardinality/testdata/cardinality_suite_in.json @@ -0,0 +1,436 @@ +[ + { + "name": "TestCollationColumnEstimate", + "cases": [ + "show stats_topn", + "explain select * from t where a='aÄa'", + "explain select * from t where a > 'aÄa'" + ] + }, + { + "Name": "TestOutOfRangeEstimation", + "Cases": [ + { + "Start": 800, + "End": 900 + }, + { + "Start": 900, + "End": 950 + }, + { + "Start": 950, + "End": 1000 + }, + { + "Start": 1000, + "End": 1050 + }, + { + "Start": 1050, + "End": 1100 + }, + { + "Start": 1150, + "End": 1200 + }, + { + "Start": 1200, + "End": 1300 + }, + { + "Start": 1300, + "End": 1400 + }, + { + "Start": 1400, + "End": 1500 + }, + { + "Start": 1500, + "End": 1600 + }, + { + "Start": 300, + "End": 899 + }, + { + "Start": 800, + "End": 1000 + }, + { + "Start": 900, + "End": 1500 + }, + { + "Start": 300, + "End": 1500 + }, + { + "Start": 200, + "End": 300 + }, + { + "Start": 100, + "End": 200 + }, + { + "Start": 200, + "End": 400 + }, + { + "Start": 200, + "End": 1000 + }, + { + "Start": 0, + "End": 100 + }, + { + "Start": -100, + "End": 100 + }, + { + "Start": -100, + "End": 0 + } + ] + }, + { + "name": "TestOutOfRangeEstimationAfterDelete", + "cases": [ + "explain format = 'brief' select * from t where a <= 300", + "explain format = 'brief' select * from t where a < 300", + "explain format = 'brief' select * from t where a <= 500", + "explain format = 'brief' select * from t where a >= 300 and a <= 900", + "explain format = 'brief' select * from t where a >= 900", + "explain format = 'brief' select * from t where a > 900", + "explain format = 'brief' select * from t where a >= 300", + "explain format = 'brief' select * from t where a <= 900", + "explain format = 'brief' select * from t where a > 800 and a < 1000", + "explain format = 'brief' select * from t where a > 900 and a < 1000", + "explain format = 'brief' select * from t where a > 900 and a < 1100", + "explain format = 'brief' select * from t where a > 200 and a < 300", + "explain format = 'brief' select * from t where a > 100 and a < 300" + ] + }, + { + "name": "TestPrimaryKeySelectivity", + "cases": [ + [ + "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" + ] + ] + }, + { + "name": "TestColumnIndexNullEstimation", + "cases": [ + "explain select b from t where b is null", + "explain select b from t where b is not null", + "explain select b from t where b is null or b > 3", + "explain select b from t use index(idx_b)", + "explain select b from t where b < 4", + "explain select * from t where a is null", + "explain select * from t where a is not null", + "explain select * from t where a is null or a > 3", + "explain select * from t", + "explain select * from t where a < 4" + ] + }, + { + "name": "TestUniqCompEqualEst", + "cases": [ + "explain select * from t where a = 1 and b = 5 and 1 = 1" + ] + }, + { + "name": "TestDiscreteDistribution", + "cases": [ + "explain select * from t where a = 'tw' and b < 0" + ] + }, + { + "name": "TestSelectCombinedLowBound", + "cases": [ + "explain select * from t where kid = 1" + ] + }, + { + "name": "TestDNFCondSelectivity", + "cases": [ + "select * from t where b > 7 or c < 4", + "select * from t where d < 5 or b > 6", + "select * from t where a > 8 or d < 4 or c > 7 or b < 5", + "select * from t where a < 8 and (b > 10 or c < 3 or b > 4) and a > 2" + ] + }, + { + "Name": "TestSmallRangeEstimation", + "Cases": [ + { + "Start": 5, + "End": 5 + }, + { + "Start": 5, + "End": 6 + }, + { + "Start": 5, + "End": 10 + }, + { + "Start": 5, + "End": 15 + }, + { + "Start": 10, + "End": 15 + }, + { + "Start": 5, + "End": 25 + }, + { + "Start": 25, + "End": 25 + } + ] + }, + { + "name": "TestDefaultSelectivityForStrMatch", + "cases": [ + "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" + ] + }, + { + "name": "TestTopNAssistedEstimationWithoutNewCollation", + "cases": [ + "explain format = 'brief' select * from t where a like '%111%'", + "explain format = 'brief' select * from t where a not like '%111%'", + "explain format = 'brief' select * from t where a regexp '.*111.*'", + "explain format = 'brief' select * from t where a not regexp '.*111.*'", + "explain format = 'brief' select * from t where ifnull(a, '1111') like '%111%'", + "explain format = 'brief' select * from t where f like '%111%'", + "explain format = 'brief' select * from t where f not like '%111%'", + "explain format = 'brief' select * from t where f regexp '.*111.*'", + "explain format = 'brief' select * from t where f not regexp '.*111.*'", + "explain format = 'brief' select * from t where ifnull(f, '1111') like '%111%'", + "explain format = 'brief' select * from t where a like '%111%' and f rlike '.*111.*'", + "explain format = 'brief' select * from t where a like '%111%' or f like '%111%'", + "explain format = 'brief' select * from t where b like '%111%'", + "explain format = 'brief' select * from t where b not like '%111%'", + "explain format = 'brief' select * from t where b regexp '.*111.*'", + "explain format = 'brief' select * from t where b not regexp '.*111.*'", + "explain format = 'brief' select * from t where ifnull(b, '1111') like '%111%'", + "explain format = 'brief' select * from t where b like '%111%' and f like '%111%'", + "explain format = 'brief' select * from t where b like '%111%' or f like '%111%'", + "explain format = 'brief' select * from t where c like '%111%'", + "explain format = 'brief' select * from t where c not like '%111%'", + "explain format = 'brief' select * from t where ifnull(c, '1111') like '%111%'", + "explain format = 'brief' select * from t where d like '%111%'", + "explain format = 'brief' select * from t where d not like '%111%'", + "explain format = 'brief' select * from t where ifnull(c, '1111') like '%111%'", + "explain format = 'brief' select * from t where e like '%111%'", + "explain format = 'brief' select * from t where e not like '%111%'", + "explain format = 'brief' select * from t where ifnull(e, '1111') like '%111%'" + ] + }, + { + "name": "TestTopNAssistedEstimationWithNewCollation", + "cases": [ + "explain format = 'brief' select * from t where a like '%111%'", + "explain format = 'brief' select * from t where a not like '%111%'", + "explain format = 'brief' select * from t where a regexp '.*111.*'", + "explain format = 'brief' select * from t where a not regexp '.*111.*'", + "explain format = 'brief' select * from t where ifnull(a, '1111') like '%111%'", + "explain format = 'brief' select * from t where f like '%111%'", + "explain format = 'brief' select * from t where f not like '%111%'", + "explain format = 'brief' select * from t where f regexp '.*111.*'", + "explain format = 'brief' select * from t where f not regexp '.*111.*'", + "explain format = 'brief' select * from t where ifnull(f, '1111') like '%111%'", + "explain format = 'brief' select * from t where a like '%111%' and f rlike '.*111.*'", + "explain format = 'brief' select * from t where a like '%111%' or f like '%111%'", + "explain format = 'brief' select * from t where b like '%111%'", + "explain format = 'brief' select * from t where b not like '%111%'", + "explain format = 'brief' select * from t where b regexp '.*111.*'", + "explain format = 'brief' select * from t where b not regexp '.*111.*'", + "explain format = 'brief' select * from t where ifnull(b, '1111') like '%111%'", + "explain format = 'brief' select * from t where b like '%111%' and f like '%111%'", + "explain format = 'brief' select * from t where b like '%111%' or f like '%111%'", + "explain format = 'brief' select * from t where c like '%111%'", + "explain format = 'brief' select * from t where c not like '%111%'", + "explain format = 'brief' select * from t where ifnull(c, '1111') like '%111%'", + "explain format = 'brief' select * from t where d like '%111%'", + "explain format = 'brief' select * from t where d not like '%111%'", + "explain format = 'brief' select * from t where ifnull(c, '1111') like '%111%'", + "explain format = 'brief' select * from t where e like '%111%'", + "explain format = 'brief' select * from t where e not like '%111%'", + "explain format = 'brief' select * from t where ifnull(e, '1111') like '%111%'" + ] + }, + { + "name": "TestGlobalStatsOutOfRangeEstimationAfterDelete", + "cases": [ + "explain format = 'brief' select * from t where a <= 300", + "explain format = 'brief' select * from t where a < 300", + "explain format = 'brief' select * from t where a <= 500", + "explain format = 'brief' select * from t where a >= 300 and a <= 900", + "explain format = 'brief' select * from t where a >= 900", + "explain format = 'brief' select * from t where a > 900", + "explain format = 'brief' select * from t where a >= 300", + "explain format = 'brief' select * from t where a <= 900", + "explain format = 'brief' select * from t where a > 800 and a < 1000", + "explain format = 'brief' select * from t where a > 900 and a < 1000", + "explain format = 'brief' select * from t where a > 900 and a < 1100", + "explain format = 'brief' select * from t where a > 200 and a < 300", + "explain format = 'brief' select * from t where a > 100 and a < 300" + ] + }, + { + "name": "TestTopNOutOfHist", + "cases": [ + "show stats_topn", + "show stats_buckets", + "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" + ] + }, + { + "name": "TestStatsVer2", + "cases": [ + "show stats_topn", + "show stats_buckets", + "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" + ] + }, + { + "name": "TestOrderingIdxSelectivityThreshold", + "cases": [ + "set @@tidb_opt_ordering_index_selectivity_threshold = 0", + "explain format = 'brief' select * from t where b >= 9950 order by c limit 1", + "explain format = 'brief' select * from t where b >= 9950 order by c desc limit 1", + "explain format = 'brief' select * from t where b >= 8999 order by c limit 1", + "explain format = 'brief' select * from t where b >= 9000 order by c limit 1", + "explain format = 'brief' select * from t where b >= 9001 order by c limit 1", + "explain format = 'brief' select * from t where a < 10001 order by c limit 1", + "explain format = 'brief' select * from t where a < 10000 order by c limit 1", + "explain format = 'brief' select * from t where a < 9999 order by c limit 1", + "explain format = 'brief' select * from t where b >= 0 and b <= 100 or c >= 0 and c <= 100 order by c limit 1", + "set @@tidb_opt_ordering_index_selectivity_threshold = 0.1", + "explain format = 'brief' select * from t where b >= 9950 order by c limit 1", + "explain format = 'brief' select * from t where b >= 9950 order by c desc limit 1", + "explain format = 'brief' select * from t where b >= 8999 order by c limit 1", + "explain format = 'brief' select * from t where b >= 9000 order by c limit 1", + "explain format = 'brief' select * from t where b >= 9001 order by c limit 1", + "explain format = 'brief' select * from t where a < 10001 order by c limit 1", + "explain format = 'brief' select * from t where a < 10000 order by c limit 1", + "explain format = 'brief' select * from t where a < 9999 order by c limit 1", + "explain format = 'brief' select * from t where b >= 0 and b <= 50 or c >= 0 and c <= 50 order by c limit 1", + "explain format = 'brief' select * from t where b >= 9950 and c >= 9950 order by c limit 1", + "explain format = 'brief' select * from t where b >= 9950 and c >= 9900 order by c limit 1" + ] + } +] diff --git a/planner/cardinality/testdata/cardinality_suite_out.json b/planner/cardinality/testdata/cardinality_suite_out.json new file mode 100644 index 0000000000000..e6a032060050d --- /dev/null +++ b/planner/cardinality/testdata/cardinality_suite_out.json @@ -0,0 +1,1956 @@ +[ + { + "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": 752.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": 4498.5 + }, + { + "Start": 800, + "End": 1000, + "Count": 1201.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": 1211.5288209899081 + }, + { + "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.13 root data:Selection", + "└─Selection 793.13 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.12 root data:Selection", + "└─Selection 458.12 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.49 root data:Selection", + "└─Selection 832.49 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 458.12 root data:Selection", + "└─Selection 458.12 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 832.49 root data:Selection", + "└─Selection 832.49 cop[tikv] gt(test.t.a, 100), lt(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + } + ] + }, + { + "Name": "TestPrimaryKeySelectivity", + "Cases": [ + [ + "TableReader_7 3333.33 root data:Selection_6", + "└─Selection_6 3333.33 cop[tikv] gt(test.t.a, \"t\")", + " └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + [ + "TableReader_6 3333.33 root data:TableRangeScan_5", + "└─TableRangeScan_5 3333.33 cop[tikv] table:t range:(1,+inf], keep order:false, stats:pseudo" + ] + ] + }, + { + "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": "TestDiscreteDistribution", + "Cases": [ + [ + "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" + ] + ] + }, + { + "Name": "TestSelectCombinedLowBound", + "Cases": [ + [ + "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" + ] + ] + }, + { + "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": "TestDefaultSelectivityForStrMatch", + "Cases": [ + { + "SQL": "set @@tidb_default_string_match_selectivity = 0.8", + "Result": null + }, + { + "SQL": "explain format = 'brief' select * from t where a + 10 > 100", + "Result": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%test%'", + "Result": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tikv] like(test.t.b, \"%test%\", 92)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not like '%test%'", + "Result": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tikv] not(like(test.t.b, \"%test%\", 92))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b regexp '.*test.*'", + "Result": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tikv] regexp(test.t.b, \".*test.*\")", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not regexp '.*test.*'", + "Result": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tikv] not(istrue_with_null(regexp(test.t.b, \".*test.*\")))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100", + "Result": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100), like(test.t.b, \"%test%\", 92)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100", + "Result": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100), not(like(test.t.b, \"%test%\", 92))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100", + "Result": [ + "TableReader 9600.00 root data:Selection", + "└─Selection 9600.00 cop[tikv] or(like(test.t.b, \"%test%\", 92), gt(plus(test.t.a, 10), 100))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "set @@tidb_default_string_match_selectivity = 0.9", + "Result": null + }, + { + "SQL": "explain format = 'brief' select * from t where a + 10 > 100", + "Result": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%test%'", + "Result": [ + "TableReader 9000.00 root data:Selection", + "└─Selection 9000.00 cop[tikv] like(test.t.b, \"%test%\", 92)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not like '%test%'", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] not(like(test.t.b, \"%test%\", 92))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b regexp '.*test.*'", + "Result": [ + "TableReader 9000.00 root data:Selection", + "└─Selection 9000.00 cop[tikv] regexp(test.t.b, \".*test.*\")", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not regexp '.*test.*'", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] not(istrue_with_null(regexp(test.t.b, \".*test.*\")))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100", + "Result": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100), like(test.t.b, \"%test%\", 92)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] gt(plus(test.t.a, 10), 100), not(like(test.t.b, \"%test%\", 92))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100", + "Result": [ + "TableReader 9800.00 root data:Selection", + "└─Selection 9800.00 cop[tikv] or(like(test.t.b, \"%test%\", 92), gt(plus(test.t.a, 10), 100))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "set @@tidb_default_string_match_selectivity = 0.1", + "Result": null + }, + { + "SQL": "explain format = 'brief' select * from t where a + 10 > 100", + "Result": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%test%'", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] like(test.t.b, \"%test%\", 92)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%test%' is true", + "Result": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tikv] istrue(like(test.t.b, \"%test%\", 92))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not like '%test%'", + "Result": [ + "TableReader 9000.00 root data:Selection", + "└─Selection 9000.00 cop[tikv] not(like(test.t.b, \"%test%\", 92))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b regexp '.*test.*'", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] regexp(test.t.b, \".*test.*\")", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b rlike '.*test.*'", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] regexp(test.t.b, \".*test.*\")", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not rlike '.*test.*'", + "Result": [ + "TableReader 9000.00 root data:Selection", + "└─Selection 9000.00 cop[tikv] not(istrue_with_null(regexp(test.t.b, \".*test.*\")))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not regexp '.*test.*'", + "Result": [ + "TableReader 9000.00 root data:Selection", + "└─Selection 9000.00 cop[tikv] not(istrue_with_null(regexp(test.t.b, \".*test.*\")))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] gt(plus(test.t.a, 10), 100), like(test.t.b, \"%test%\", 92)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100", + "Result": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100), not(like(test.t.b, \"%test%\", 92))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100", + "Result": [ + "TableReader 8200.00 root data:Selection", + "└─Selection 8200.00 cop[tikv] or(like(test.t.b, \"%test%\", 92), gt(plus(test.t.a, 10), 100))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "set @@tidb_default_string_match_selectivity = 0", + "Result": null + }, + { + "SQL": "explain format = 'brief' select * from t where a + 10 > 100", + "Result": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%test%'", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] like(test.t.b, \"%test%\", 92)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not like '%test%'", + "Result": [ + "TableReader 9000.00 root data:Selection", + "└─Selection 9000.00 cop[tikv] not(like(test.t.b, \"%test%\", 92))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b regexp '.*test.*'", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] regexp(test.t.b, \".*test.*\")", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not regexp '.*test.*'", + "Result": [ + "TableReader 9000.00 root data:Selection", + "└─Selection 9000.00 cop[tikv] not(istrue_with_null(regexp(test.t.b, \".*test.*\")))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] gt(plus(test.t.a, 10), 100), like(test.t.b, \"%test%\", 92)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100", + "Result": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100), not(like(test.t.b, \"%test%\", 92))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100", + "Result": [ + "TableReader 8200.00 root data:Selection", + "└─Selection 8200.00 cop[tikv] or(like(test.t.b, \"%test%\", 92), gt(plus(test.t.a, 10), 100))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "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": "TestTopNOutOfHist", + "Cases": [ + [ + "test topn_before_hist a 0 1 4", + "test topn_before_hist a 0 3 2", + "test topn_before_hist idx 1 1 4", + "test topn_before_hist idx 1 3 2", + "test topn_after_hist a 0 2 2", + "test topn_after_hist a 0 7 4", + "test topn_after_hist idx 1 2 2", + "test topn_after_hist idx 1 7 4", + "test topn_before_hist_no_index a 0 1 4", + "test topn_before_hist_no_index a 0 3 2", + "test topn_after_hist_no_index a 0 2 2", + "test topn_after_hist_no_index a 0 7 4" + ], + [ + "test topn_before_hist a 0 0 2 1 4 5 0", + "test topn_before_hist a 0 1 3 1 6 6 0", + "test topn_before_hist idx 1 0 2 1 4 5 0", + "test topn_before_hist idx 1 1 3 1 6 6 0", + "test topn_after_hist a 0 0 2 1 3 4 0", + "test topn_after_hist a 0 1 3 1 5 5 0", + "test topn_after_hist idx 1 0 2 1 3 4 0", + "test topn_after_hist idx 1 1 3 1 5 5 0", + "test topn_before_hist_no_index a 0 0 2 1 4 5 0", + "test topn_before_hist_no_index a 0 1 3 1 6 6 0", + "test topn_after_hist_no_index a 0 0 2 1 3 4 0", + "test topn_after_hist_no_index a 0 1 3 1 5 5 0" + ], + [ + "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" + ], + [ + "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" + ], + [ + "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" + ], + [ + "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" + ], + [ + "TableReader_7 4.00 root data:Selection_6", + "└─Selection_6 4.00 cop[tikv] eq(test.topn_after_hist_no_index.a, 7)", + " └─TableFullScan_5 9.00 cop[tikv] table:topn_after_hist_no_index keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.topn_after_hist_no_index.a, 6)", + " └─TableFullScan_5 9.00 cop[tikv] table:topn_after_hist_no_index keep order:false" + ], + [ + "TableReader_7 4.00 root data:Selection_6", + "└─Selection_6 4.00 cop[tikv] eq(test.topn_before_hist_no_index.a, 1)", + " └─TableFullScan_5 9.00 cop[tikv] table:topn_before_hist_no_index keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.topn_before_hist_no_index.a, 2)", + " └─TableFullScan_5 9.00 cop[tikv] table:topn_before_hist_no_index keep order:false" + ] + ] + }, + { + "Name": "TestStatsVer2", + "Cases": [ + [ + "test tint a 0 1 1", + "test tint a 0 2 1", + "test tint b 0 1 1", + "test tint b 0 2 1", + "test tint c 0 1 1", + "test tint c 0 2 1", + "test tint singular 1 1 1", + "test tint singular 1 2 1", + "test tint multi 1 (1, 1) 1", + "test tint multi 1 (2, 2) 1", + "test tdouble a 0 1 1", + "test tdouble a 0 2 1", + "test tdouble b 0 1 1", + "test tdouble b 0 2 1", + "test tdouble c 0 1 1", + "test tdouble c 0 2 1", + "test tdouble singular 1 1 1", + "test tdouble singular 1 2 1", + "test tdouble multi 1 (1, 1) 1", + "test tdouble multi 1 (2, 2) 1", + "test tdecimal a 0 1.00000000000000000000 1", + "test tdecimal a 0 2.00000000000000000000 1", + "test tdecimal b 0 1.00000000000000000000 1", + "test tdecimal b 0 2.00000000000000000000 1", + "test tdecimal c 0 1.00000000000000000000 1", + "test tdecimal c 0 2.00000000000000000000 1", + "test tdecimal singular 1 1.00000000000000000000 1", + "test tdecimal singular 1 2.00000000000000000000 1", + "test tdecimal multi 1 (1.00000000000000000000, 1.00000000000000000000) 1", + "test tdecimal multi 1 (2.00000000000000000000, 2.00000000000000000000) 1", + "test tstring a 0 1 1", + "test tstring a 0 2 1", + "test tstring b 0 1 1", + "test tstring b 0 2 1", + "test tstring c 0 1 1", + "test tstring c 0 2 1", + "test tstring singular 1 1 1", + "test tstring singular 1 2 1", + "test tstring multi 1 (1, 1) 1", + "test tstring multi 1 (2, 2) 1", + "test tdatetime a 0 2001-01-01 00:00:00 1", + "test tdatetime a 0 2001-01-02 00:00:00 1", + "test tdatetime b 0 2001-01-01 00:00:00 1", + "test tdatetime b 0 2001-01-02 00:00:00 1", + "test tdatetime c 0 2001-01-01 00:00:00 1", + "test tdatetime c 0 2001-01-02 00:00:00 1", + "test tdatetime singular 1 2001-01-01 00:00:00 1", + "test tdatetime singular 1 2001-01-02 00:00:00 1", + "test tdatetime multi 1 (2001-01-01 00:00:00, 2001-01-01 00:00:00) 1", + "test tdatetime multi 1 (2001-01-02 00:00:00, 2001-01-02 00:00:00) 1", + "test tprefix a 0 111 1", + "test tprefix a 0 222 1", + "test tprefix b 0 111 1", + "test tprefix b 0 222 1", + "test tprefix prefixa 1 11 1", + "test tprefix prefixa 1 22 1", + "test ct1 a 0 1 1", + "test ct1 a 0 2 1", + "test ct1 pk 0 1 1", + "test ct1 pk 0 2 1", + "test ct1 PRIMARY 1 1 1", + "test ct1 PRIMARY 1 2 1", + "test ct2 a 0 1 1", + "test ct2 a 0 2 1", + "test ct2 b 0 1 1", + "test ct2 b 0 2 1", + "test ct2 c 0 1 1", + "test ct2 c 0 2 1", + "test ct2 PRIMARY 1 (1, 1) 1", + "test ct2 PRIMARY 1 (2, 2) 1" + ], + [ + "test tint a 0 0 3 1 3 5 0", + "test tint a 0 1 6 1 6 8 0", + "test tint b 0 0 3 1 3 5 0", + "test tint b 0 1 6 1 6 8 0", + "test tint c 0 0 3 1 3 5 0", + "test tint c 0 1 6 1 6 8 0", + "test tint singular 1 0 3 1 3 5 0", + "test tint singular 1 1 6 1 6 8 0", + "test tint multi 1 0 3 1 (3, 3) (5, 5) 0", + "test tint multi 1 1 6 1 (6, 6) (8, 8) 0", + "test tdouble a 0 0 3 1 3 5 0", + "test tdouble a 0 1 6 1 6 8 0", + "test tdouble b 0 0 3 1 3 5 0", + "test tdouble b 0 1 6 1 6 8 0", + "test tdouble c 0 0 3 1 3 5 0", + "test tdouble c 0 1 6 1 6 8 0", + "test tdouble singular 1 0 3 1 3 5 0", + "test tdouble singular 1 1 6 1 6 8 0", + "test tdouble multi 1 0 3 1 (3, 3) (5, 5) 0", + "test tdouble multi 1 1 6 1 (6, 6) (8, 8) 0", + "test tdecimal a 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0", + "test tdecimal a 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0", + "test tdecimal b 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0", + "test tdecimal b 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0", + "test tdecimal c 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0", + "test tdecimal c 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0", + "test tdecimal singular 1 0 3 1 3.00000000000000000000 5.00000000000000000000 0", + "test tdecimal singular 1 1 6 1 6.00000000000000000000 8.00000000000000000000 0", + "test tdecimal multi 1 0 3 1 (3.00000000000000000000, 3.00000000000000000000) (5.00000000000000000000, 5.00000000000000000000) 0", + "test tdecimal multi 1 1 6 1 (6.00000000000000000000, 6.00000000000000000000) (8.00000000000000000000, 8.00000000000000000000) 0", + "test tstring a 0 0 3 1 3 5 0", + "test tstring a 0 1 6 1 6 8 0", + "test tstring b 0 0 3 1 3 5 0", + "test tstring b 0 1 6 1 6 8 0", + "test tstring c 0 0 3 1 3 5 0", + "test tstring c 0 1 6 1 6 8 0", + "test tstring singular 1 0 3 1 3 5 0", + "test tstring singular 1 1 6 1 6 8 0", + "test tstring multi 1 0 3 1 (3, 3) (5, 5) 0", + "test tstring multi 1 1 6 1 (6, 6) (8, 8) 0", + "test tdatetime a 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0", + "test tdatetime a 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0", + "test tdatetime b 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0", + "test tdatetime b 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0", + "test tdatetime c 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0", + "test tdatetime c 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0", + "test tdatetime singular 1 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0", + "test tdatetime singular 1 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0", + "test 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", + "test 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", + "test tprefix a 0 0 2 1 333 444 0", + "test tprefix a 0 1 4 1 555 666 0", + "test tprefix b 0 0 2 1 333 444 0", + "test tprefix b 0 1 4 1 555 666 0", + "test tprefix prefixa 1 0 2 1 33 44 0", + "test tprefix prefixa 1 1 4 1 55 66 0", + "test ct1 a 0 0 3 1 3 5 0", + "test ct1 a 0 1 6 1 6 8 0", + "test ct1 pk 0 0 3 1 3 5 0", + "test ct1 pk 0 1 6 1 6 8 0", + "test ct1 PRIMARY 1 0 3 1 3 5 0", + "test ct1 PRIMARY 1 1 6 1 6 8 0", + "test ct2 a 0 0 3 1 3 5 0", + "test ct2 a 0 1 6 1 6 8 0", + "test ct2 b 0 0 3 1 3 5 0", + "test ct2 b 0 1 6 1 6 8 0", + "test ct2 c 0 0 3 1 3 5 0", + "test ct2 c 0 1 6 1 6 8 0", + "test ct2 PRIMARY 1 0 3 1 (3, 3) (5, 5) 0", + "test ct2 PRIMARY 1 1 6 1 (6, 6) (8, 8) 0" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tint.a, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tint.a, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tint.a, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.a, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.a, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.a, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.a, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.a, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.a, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tstring.a, \"1\")", + " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tstring.a, \"4\")", + " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tstring.a, \"8\")", + " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.a, 2001-01-01 00:00:00.000000)", + " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.a, 2001-01-02 00:00:00.000000)", + " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.a, 2001-01-04 00:00:00.000000)", + " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tprefix.a, \"111\")", + " └─TableFullScan_5 6.00 cop[tikv] table:tprefix keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tprefix.a, \"444\")", + " └─TableFullScan_5 6.00 cop[tikv] table:tprefix keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tprefix.a, \"888\")", + " └─TableFullScan_5 6.00 cop[tikv] table:tprefix keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tint.b, 1), eq(test.tint.c, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tint.b, 4), eq(test.tint.c, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tint.b, 8), eq(test.tint.c, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 1), eq(test.tdouble.c, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 4), eq(test.tdouble.c, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 8), eq(test.tdouble.c, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 1), eq(test.tdecimal.c, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 4), eq(test.tdecimal.c, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 8), eq(test.tdecimal.c, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tstring.b, \"1\"), eq(test.tstring.c, \"1\")", + " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tstring.b, \"4\"), eq(test.tstring.c, \"4\")", + " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tstring.b, \"8\"), eq(test.tstring.c, \"8\")", + " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.b, 2001-01-01 00:00:00.000000), eq(test.tdatetime.c, 2001-01-01 00:00:00.000000)", + " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.b, 2001-01-02 00:00:00.000000), eq(test.tdatetime.c, 2001-01-02 00:00:00.000000)", + " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.b, 2001-01-04 00:00:00.000000), eq(test.tdatetime.c, 2001-01-04 00:00:00.000000)", + " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tint.b, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tint.b, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tint.b, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 1)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 4)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 8)", + " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tstring.b, \"1\")", + " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tstring.b, \"4\")", + " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tstring.b, \"8\")", + " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.b, 2001-01-01 00:00:00.000000)", + " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.b, 2001-01-02 00:00:00.000000)", + " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.b, 2001-01-04 00:00:00.000000)", + " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" + ], + [ + "TableReader_6 5.00 root data:TableRangeScan_5", + "└─TableRangeScan_5 5.00 cop[tikv] table:ct1 range:[\"1\",\"4\"], keep order:false" + ], + [ + "TableReader_6 3.75 root data:TableRangeScan_5", + "└─TableRangeScan_5 3.75 cop[tikv] table:ct1 range:[\"4\",\"6\"], keep order:false" + ], + [ + "TableReader_6 3.00 root data:TableRangeScan_5", + "└─TableRangeScan_5 3.00 cop[tikv] table:ct1 range:[\"6\",\"8\"], keep order:false" + ], + [ + "TableReader_6 1.00 root data:TableRangeScan_5", + "└─TableRangeScan_5 1.00 cop[tikv] table:ct2 range:[1 1,1 8], keep order:false" + ], + [ + "TableReader_6 1.00 root data:TableRangeScan_5", + "└─TableRangeScan_5 1.00 cop[tikv] table:ct2 range:[4 1,4 8], keep order:false" + ], + [ + "TableReader_6 1.00 root data:TableRangeScan_5", + "└─TableRangeScan_5 1.00 cop[tikv] table:ct2 range:[8 1,8 8], keep order:false" + ] + ] + }, + { + "Name": "TestOrderingIdxSelectivityThreshold", + "Cases": [ + { + "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": "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" + ] + } + ] + } +] diff --git a/planner/cardinality/trace.go b/planner/cardinality/trace.go new file mode 100644 index 0000000000000..72f57b58169a5 --- /dev/null +++ b/planner/cardinality/trace.go @@ -0,0 +1,110 @@ +// 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" + "errors" + + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/format" + "github.com/pingcap/tidb/sessionctx" + driver "github.com/pingcap/tidb/types/parser_driver" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/tracing" + "go.uber.org/zap" +) + +// CETraceExpr appends an expression and related information into CE trace +func CETraceExpr(sctx sessionctx.Context, tableID int64, tp string, expr expression.Expression, rowCount float64) { + exprStr, err := ExprToString(expr) + if err != nil { + logutil.BgLogger().Debug("Failed to trace CE of an expression", zap.String("category", "OptimizerTrace"), + zap.Any("expression", expr)) + return + } + rec := tracing.CETraceRecord{ + TableID: tableID, + Type: tp, + Expr: exprStr, + RowCount: uint64(rowCount), + } + sc := sctx.GetSessionVars().StmtCtx + sc.OptimizerCETrace = append(sc.OptimizerCETrace, &rec) +} + +// ExprToString prints an Expression into a string which can appear in a SQL. +// +// It might be too tricky because it makes use of TiDB allowing using internal function name in SQL. +// For example, you can write `eq`(a, 1), which is the same as a = 1. +// We should have implemented this by first implementing a method to turn an expression to an AST +// +// then call astNode.Restore(), like the Constant case here. But for convenience, we use this trick for now. +// +// It may be more appropriate to put this in expression package. But currently we only use it for CE trace, +// +// and it may not be general enough to handle all possible expressions. So we put it here for now. +func ExprToString(e expression.Expression) (string, error) { + switch expr := e.(type) { + case *expression.ScalarFunction: + var buffer bytes.Buffer + buffer.WriteString("`" + expr.FuncName.L + "`(") + switch expr.FuncName.L { + case ast.Cast: + for _, arg := range expr.GetArgs() { + argStr, err := ExprToString(arg) + if err != nil { + return "", err + } + buffer.WriteString(argStr) + buffer.WriteString(", ") + buffer.WriteString(expr.RetType.String()) + } + default: + for i, arg := range expr.GetArgs() { + argStr, err := ExprToString(arg) + if err != nil { + return "", err + } + buffer.WriteString(argStr) + if i+1 != len(expr.GetArgs()) { + buffer.WriteString(", ") + } + } + } + buffer.WriteString(")") + return buffer.String(), nil + case *expression.Column: + return expr.String(), nil + case *expression.CorrelatedColumn: + return "", errors.New("tracing for correlated columns not supported now") + case *expression.Constant: + value, err := expr.Eval(chunk.Row{}) + if err != nil { + return "", err + } + valueExpr := driver.ValueExpr{Datum: value} + var buffer bytes.Buffer + restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, &buffer) + err = valueExpr.Restore(restoreCtx) + if err != nil { + return "", err + } + return buffer.String(), nil + } + return "", errors.New("unexpected type of Expression") +} diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index c17af4b188ea6..d0fffee8402ba 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -106,6 +106,7 @@ go_library( "//parser/opcode", "//parser/terror", "//parser/types", + "//planner/cardinality", "//planner/core/internal", "//planner/core/internal/base", "//planner/core/metrics", diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index ab48af2635d67..750d9e60e359d 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/cardinality" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/planner/util/fixcontrol" @@ -1078,7 +1079,7 @@ func (p *LogicalJoin) constructInnerTableScanTask( countAfterAccess := rowCount if len(ts.filterCondition) > 0 { var err error - selectivity, _, err = ds.tableStats.HistColl.Selectivity(ds.SCtx(), ts.filterCondition, ds.possibleAccessPaths) + selectivity, _, err = cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, ts.filterCondition, ds.possibleAccessPaths) if err != nil || selectivity <= 0 { logutil.BgLogger().Debug("unexpected selectivity, use selection factor", zap.Float64("selectivity", selectivity), zap.String("table", ts.TableAsName.L)) selectivity = SelectionFactor @@ -1363,7 +1364,7 @@ func (p *LogicalJoin) constructInnerIndexScanTask( } // Assume equal conditions used by index join and other conditions are independent. if len(tblConds) > 0 { - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), tblConds, ds.possibleAccessPaths) + selectivity, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, tblConds, ds.possibleAccessPaths) if err != nil || selectivity <= 0 { logutil.BgLogger().Debug("unexpected selectivity, use selection factor", zap.Float64("selectivity", selectivity), zap.String("table", ds.TableAsName.L)) selectivity = SelectionFactor @@ -1381,7 +1382,7 @@ func (p *LogicalJoin) constructInnerIndexScanTask( tmpPath.CountAfterAccess = cnt } if len(indexConds) > 0 { - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), indexConds, ds.possibleAccessPaths) + selectivity, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, indexConds, ds.possibleAccessPaths) if err != nil || selectivity <= 0 { logutil.BgLogger().Debug("unexpected selectivity, use selection factor", zap.Float64("selectivity", selectivity), zap.String("table", ds.TableAsName.L)) selectivity = SelectionFactor diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index b4525849f047a..3ee365484d3f1 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/cardinality" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" @@ -1399,7 +1400,7 @@ func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, } ts.filterCondition = newFilterConds if len(ts.filterCondition) > 0 { - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), ts.filterCondition, nil) + selectivity, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, ts.filterCondition, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = SelectionFactor @@ -1492,7 +1493,7 @@ func (ds *DataSource) buildIndexMergeTableScan(tableFilters []expression.Express pushedFilters = pushedFilters1 remainingFilters = append(remainingFilters, remainingFilters1...) if len(pushedFilters) != 0 { - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), pushedFilters, nil) + selectivity, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, pushedFilters, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = SelectionFactor @@ -1902,7 +1903,7 @@ func (is *PhysicalIndexScan) addPushedDownSelection(copTask *copTask, p *DataSou copTask.finishIndexPlan() tableSel := PhysicalSelection{Conditions: tableConds}.Init(is.SCtx(), finalStats, is.SelectBlockOffset()) if len(copTask.rootTaskConds) != 0 { - selectivity, _, err := copTask.tblColHists.Selectivity(is.SCtx(), tableConds, nil) + selectivity, _, err := cardinality.Selectivity(is.SCtx(), copTask.tblColHists, tableConds, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = SelectionFactor @@ -2618,7 +2619,7 @@ func (ts *PhysicalTableScan) addPushedDownSelection(copTask *copTask, stats *pro if len(ts.filterCondition) > 0 { sel := PhysicalSelection{Conditions: ts.filterCondition}.Init(ts.SCtx(), stats, ts.SelectBlockOffset()) if len(copTask.rootTaskConds) != 0 { - selectivity, _, err := copTask.tblColHists.Selectivity(ts.SCtx(), ts.filterCondition, nil) + selectivity, _, err := cardinality.Selectivity(ts.SCtx(), copTask.tblColHists, ts.filterCondition, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = SelectionFactor diff --git a/planner/core/indexmerge_path.go b/planner/core/indexmerge_path.go index e2b56f2bd1b86..2c22eae46f660 100644 --- a/planner/core/indexmerge_path.go +++ b/planner/core/indexmerge_path.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/cardinality" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/planner/util/debugtrace" "github.com/pingcap/tidb/sessionctx" @@ -176,7 +177,7 @@ func (ds *DataSource) generateIndexMergeOrPaths(filters []expression.Expression) } } accessDNF := expression.ComposeDNFCondition(ds.SCtx(), accessConds...) - sel, _, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), []expression.Expression{accessDNF}, nil) + sel, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, []expression.Expression{accessDNF}, nil) if err != nil { logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) sel = SelectionFactor @@ -462,7 +463,7 @@ func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int) *util.Access } // 3. Estimate the row count after partial paths. - sel, _, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), partialFilters, nil) + sel, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, partialFilters, nil) if err != nil { logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) sel = SelectionFactor diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index e158390652f9e..e5cb9c9f6eff3 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/cardinality" fd "github.com/pingcap/tidb/planner/funcdep" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" @@ -1871,7 +1872,7 @@ func (ds *DataSource) deriveIndexPathStats(path *util.AccessPath, _ []expression path.CountAfterAccess = math.Min(ds.StatsInfo().RowCount/SelectionFactor, float64(ds.statisticTable.RealtimeCount)) } if path.IndexFilters != nil { - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), path.IndexFilters, nil) + selectivity, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, path.IndexFilters, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = SelectionFactor diff --git a/planner/core/stats.go b/planner/core/stats.go index 6a247d7a5c088..7b766dfece901 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/cardinality" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/planner/util/debugtrace" @@ -320,16 +321,17 @@ func (ds *DataSource) deriveStatsByFilter(conds expression.CNFExprs, filledPaths debugtrace.EnterContextCommon(ds.SCtx()) defer debugtrace.LeaveContextCommon(ds.SCtx()) } - selectivity, nodes, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), conds, filledPaths) + selectivity, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, conds, filledPaths) if err != nil { logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) selectivity = SelectionFactor } - stats := ds.tableStats.Scale(selectivity) - if ds.SCtx().GetSessionVars().OptimizerSelectivityLevel >= 1 { - stats.HistColl = stats.HistColl.NewHistCollBySelectivity(ds.SCtx(), nodes) - } - return stats + // TODO: remove NewHistCollBySelectivity later on. + // if ds.SCtx().GetSessionVars().OptimizerSelectivityLevel >= 1 { + // Only '0' is suggested, see https://docs.pingcap.com/zh/tidb/stable/system-variables#tidb_optimizer_selectivity_level. + // stats.HistColl = stats.HistColl.NewHistCollBySelectivity(ds.SCtx(), nodes) + // } + return ds.tableStats.Scale(selectivity) } // We bind logic of derivePathStats and tryHeuristics together. When some path matches the heuristic rule, we don't need diff --git a/planner/core/task.go b/planner/core/task.go index 56b6d5c634c6e..ae758d0e242f2 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/cardinality" "github.com/pingcap/tidb/planner/core/internal/base" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" @@ -759,7 +760,7 @@ func (t *copTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { func (t *copTask) handleRootTaskConds(ctx sessionctx.Context, newTask *rootTask) { if len(t.rootTaskConds) > 0 { - selectivity, _, err := t.tblColHists.Selectivity(ctx, t.rootTaskConds, nil) + selectivity, _, err := cardinality.Selectivity(ctx, t.tblColHists, t.rootTaskConds, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = SelectionFactor @@ -2605,7 +2606,7 @@ func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { logutil.BgLogger().Error("expect Selection or TableScan for mppTask.p", zap.String("mppTask.p", t.p.TP())) return invalidTask } - selectivity, _, err := t.tblColHists.Selectivity(ctx, t.rootTaskConds, nil) + selectivity, _, err := cardinality.Selectivity(ctx, t.tblColHists, t.rootTaskConds, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = SelectionFactor diff --git a/planner/core/tiflash_selection_late_materialization.go b/planner/core/tiflash_selection_late_materialization.go index a4b7ee49d51f2..ce303624a5598 100644 --- a/planner/core/tiflash_selection_late_materialization.go +++ b/planner/core/tiflash_selection_late_materialization.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/planner/cardinality" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" @@ -135,7 +136,7 @@ func groupByColumnsSortBySelectivity(sctx sessionctx.Context, conds []expression // Estimate the selectivity of each group and check if it is larger than the selectivityThreshold var exprGroups []expressionGroup for _, group := range groupMap { - selectivity, _, err := physicalTableScan.tblColHists.Selectivity(sctx, group, nil) + selectivity, _, err := cardinality.Selectivity(sctx, physicalTableScan.tblColHists, group, nil) if err != nil { logutil.BgLogger().Warn("calculate selectivity failed, do not push down the conditions group", zap.Error(err)) continue @@ -226,7 +227,7 @@ func predicatePushDownToTableScanImpl(sctx sessionctx.Context, physicalSelection for _, exprGroup := range sortedConds { mergedConds := append(selectedConds, exprGroup.exprs...) - selectivity, _, err := physicalTableScan.tblColHists.Selectivity(sctx, mergedConds, nil) + selectivity, _, err := cardinality.Selectivity(sctx, physicalTableScan.tblColHists, mergedConds, nil) if err != nil { logutil.BgLogger().Warn("calculate selectivity failed, do not push down the conditions group", zap.Error(err)) continue diff --git a/server/testdata/optimizer_suite_out.json b/server/testdata/optimizer_suite_out.json index ccf6d0a5ef637..8c7b11e34f6db 100644 --- a/server/testdata/optimizer_suite_out.json +++ b/server/testdata/optimizer_suite_out.json @@ -171,7 +171,7 @@ }, { "github.com/pingcap/tidb/planner/core.(*DataSource).deriveStatsByFilter": { - "github.com/pingcap/tidb/statistics.(*HistColl).Selectivity": [ + "github.com/pingcap/tidb/planner/cardinality.Selectivity": [ { "Input Expressions": [ "lt(test.t.col1, 127)", @@ -386,7 +386,7 @@ "github.com/pingcap/tidb/planner/core.(*DataSource).DeriveStats": [ { "github.com/pingcap/tidb/planner/core.(*DataSource).deriveStatsByFilter": { - "github.com/pingcap/tidb/statistics.(*HistColl).Selectivity": [ + "github.com/pingcap/tidb/planner/cardinality.Selectivity": [ { "Input Expressions": [ "eq(test.t.col1, 100)" @@ -649,7 +649,7 @@ }, { "github.com/pingcap/tidb/planner/core.(*DataSource).deriveStatsByFilter": { - "github.com/pingcap/tidb/statistics.(*HistColl).Selectivity": [ + "github.com/pingcap/tidb/planner/cardinality.Selectivity": [ { "Input Expressions": [ "in(test.t.col1, 1, 2, 3)" diff --git a/statistics/BUILD.bazel b/statistics/BUILD.bazel index 9fedac82974df..3ed90c9641b3b 100644 --- a/statistics/BUILD.bazel +++ b/statistics/BUILD.bazel @@ -29,18 +29,15 @@ go_library( "//kv", "//parser/ast", "//parser/charset", - "//parser/format", "//parser/model", "//parser/mysql", "//parser/terror", - "//planner/util", "//planner/util/debugtrace", "//sessionctx", "//sessionctx/stmtctx", "//sessionctx/variable", "//tablecodec", "//types", - "//types/parser_driver", "//util/chunk", "//util/codec", "//util/collate", @@ -75,30 +72,26 @@ go_test( "main_test.go", "sample_test.go", "scalar_test.go", - "selectivity_test.go", "statistics_test.go", "trace_test.go", ], data = glob(["testdata/**"]), embed = [":statistics"], flaky = True, - shard_count = 50, + shard_count = 46, deps = [ "//config", - "//domain", "//executor", "//expression", "//infoschema", - "//kv", "//parser", "//parser/ast", "//parser/model", "//parser/mysql", + "//planner/cardinality", "//planner/core", - "//session", "//sessionctx", "//sessionctx/stmtctx", - "//sessionctx/variable", "//statistics/handle", "//testkit", "//testkit/testdata", diff --git a/statistics/integration_test.go b/statistics/integration_test.go index 0fcd201cb1a31..60b03e3fdf11e 100644 --- a/statistics/integration_test.go +++ b/statistics/integration_test.go @@ -24,14 +24,10 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/codec" "github.com/stretchr/testify/require" ) @@ -745,149 +741,6 @@ func TestUpdateNotLoadIndexFMSketch(t *testing.T) { require.Nil(t, h.GetPartitionStats(tblInfo, p1.ID).Indices[idxInfo.ID].FMSketch) } -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(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.Columns[int64(i)] = &statistics.Column{ - Histogram: *mockStatsHistogram(int64(i), colValues, 1000, types.NewFieldType(mysql.TypeLonglong)), - Info: tblInfo.Columns[i-1], - StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), - StatsVer: 2, - } - } - generateMapsForMockStatsTbl(mockStatsTbl) - stat := h.GetTableStats(tblInfo) - stat.HistColl = mockStatsTbl.HistColl - - 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" - - testKit.MustQuery(query).Check(testkit.Rows( - "IndexJoin 1000000.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) 1000000.00 root ", - " ├─Selection(Build) 500000000.00 cop[tikv] 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) 1000000.00 cop[tikv] eq(test.t.a, 0)", - " └─TableRowIDScan 500000000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - )) - - testKit.MustExec("set @@tidb_opt_fix_control = '44855:ON'") - testKit.MustQuery(query).Check(testkit.Rows( - "IndexJoin 1000000.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) 1000000.00 root ", - " ├─Selection(Build) 1000000.00 cop[tikv] 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) 1000000.00 cop[tikv] eq(test.t.a, 0)", - " └─TableRowIDScan 1000000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - )) -} - -func TestOrderingIdxSelectivityThreshold(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - h := dom.StatsHandle() - sc := &stmtctx.StatementContext{TimeZone: 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(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.Columns[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, nil, val) - require.NoError(t, err) - idxValues = append(idxValues, types.NewBytesDatum(b)) - } - - for i := 2; i <= 3; i++ { - mockStatsTbl.Columns[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.Indices[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 := statistics.GetIntegrationSuiteData() - 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 TestIssue44369(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) h := dom.StatsHandle() diff --git a/statistics/main_test.go b/statistics/main_test.go index 00897fd496feb..81571d0c64e38 100644 --- a/statistics/main_test.go +++ b/statistics/main_test.go @@ -44,7 +44,6 @@ func TestMain(m *testing.M) { }) testDataMap.LoadTestSuiteData("testdata", "integration_suite") - testDataMap.LoadTestSuiteData("testdata", "stats_suite") testDataMap.LoadTestSuiteData("testdata", "trace_suite") opts := []goleak.Option{ @@ -65,10 +64,6 @@ func GetIntegrationSuiteData() testdata.TestData { return testDataMap["integration_suite"] } -func GetStatsSuiteData() testdata.TestData { - return testDataMap["stats_suite"] -} - func GetTraceSuiteData() testdata.TestData { return testDataMap["trace_suite"] } diff --git a/statistics/selectivity.go b/statistics/selectivity.go index a5b80961c5dee..4c8c44f53d33a 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -15,34 +15,11 @@ package statistics import ( - "bytes" - "cmp" - "math" - "math/bits" - "slices" - - "github.com/pingcap/errors" - "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/parser/ast" - "github.com/pingcap/tidb/parser/format" - "github.com/pingcap/tidb/parser/mysql" - planutil "github.com/pingcap/tidb/planner/util" - "github.com/pingcap/tidb/planner/util/debugtrace" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/types" - driver "github.com/pingcap/tidb/types/parser_driver" - "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" - "github.com/pingcap/tidb/util/tracing" - "go.uber.org/zap" - "golang.org/x/exp/maps" ) -// If one condition can't be calculated, we will assume that the selectivity of this condition is 0.8. -const selectionFactor = 0.8 - // StatsNode is used for calculating selectivity. +// TODO: remove this structure after removing tidb_optimizer_selectivity_level. type StatsNode struct { // Ranges contains all the Ranges we got. Ranges []*ranger.Range @@ -65,685 +42,3 @@ const ( PkType ColType ) - -func compareType(l, r int) int { - if l == r { - return 0 - } - if l == ColType { - return -1 - } - if l == PkType { - return 1 - } - if r == ColType { - return 1 - } - return -1 -} - -// MockStatsNode is only used for test. -func MockStatsNode(id int64, m int64, num int) *StatsNode { - return &StatsNode{ID: id, mask: m, numCols: num} -} - -const unknownColumnID = math.MinInt64 - -// getConstantColumnID receives two expressions and if one of them is column and another is constant, it returns the -// ID of the column. -func getConstantColumnID(e []expression.Expression) int64 { - if len(e) != 2 { - return unknownColumnID - } - col, ok1 := e[0].(*expression.Column) - _, ok2 := e[1].(*expression.Constant) - if ok1 && ok2 { - return col.ID - } - col, ok1 = e[1].(*expression.Column) - _, ok2 = e[0].(*expression.Constant) - if ok1 && ok2 { - return col.ID - } - return unknownColumnID -} - -func pseudoSelectivity(coll *HistColl, exprs []expression.Expression) float64 { - minFactor := selectionFactor - colExists := make(map[string]bool) - for _, expr := range exprs { - fun, ok := expr.(*expression.ScalarFunction) - if !ok { - continue - } - colID := getConstantColumnID(fun.GetArgs()) - if colID == unknownColumnID { - continue - } - switch fun.FuncName.L { - case ast.EQ, ast.NullEQ, ast.In: - minFactor = math.Min(minFactor, 1.0/pseudoEqualRate) - col, ok := coll.Columns[colID] - if !ok { - continue - } - colExists[col.Info.Name.L] = true - if mysql.HasUniKeyFlag(col.Info.GetFlag()) { - return 1.0 / float64(coll.RealtimeCount) - } - case ast.GE, ast.GT, ast.LE, ast.LT: - minFactor = math.Min(minFactor, 1.0/pseudoLessRate) - // FIXME: To resolve the between case. - } - } - if len(colExists) == 0 { - return minFactor - } - // use the unique key info - for _, idx := range coll.Indices { - if !idx.Info.Unique { - continue - } - unique := true - for _, col := range idx.Info.Columns { - if !colExists[col.Name.L] { - unique = false - break - } - } - if unique { - return 1.0 / float64(coll.RealtimeCount) - } - } - return minFactor -} - -// isColEqCorCol checks if the expression is a eq function that one side is correlated column and another is column. -// If so, it will return the column's reference. Otherwise return nil instead. -func isColEqCorCol(filter expression.Expression) *expression.Column { - f, ok := filter.(*expression.ScalarFunction) - if !ok || f.FuncName.L != ast.EQ { - return nil - } - if c, ok := f.GetArgs()[0].(*expression.Column); ok { - if _, ok := f.GetArgs()[1].(*expression.CorrelatedColumn); ok { - return c - } - } - if c, ok := f.GetArgs()[1].(*expression.Column); ok { - if _, ok := f.GetArgs()[0].(*expression.CorrelatedColumn); ok { - return c - } - } - return nil -} - -// Selectivity is a function calculate the selectivity of the expressions. -// The definition of selectivity is (row count after filter / row count before filter). -// And exprs must be CNF now, in other words, `exprs[0] and exprs[1] and ... and exprs[len - 1]` should be held when you call this. -// Currently the time complexity is o(n^2). -func (coll *HistColl) Selectivity( - ctx sessionctx.Context, - exprs []expression.Expression, - filledPaths []*planutil.AccessPath, -) ( - result float64, - retStatsNodes []*StatsNode, - err error, -) { - var exprStrs []string - if ctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { - debugtrace.EnterContextCommon(ctx) - exprStrs = expression.ExprsToStringsForDisplay(exprs) - debugtrace.RecordAnyValuesWithNames(ctx, "Input Expressions", exprStrs) - defer func() { - debugtrace.RecordAnyValuesWithNames(ctx, "Result", result) - debugtrace.LeaveContextCommon(ctx) - }() - } - // If table's count is zero or conditions are empty, we should return 100% selectivity. - if coll.RealtimeCount == 0 || len(exprs) == 0 { - return 1, nil, nil - } - ret := 1.0 - sc := ctx.GetSessionVars().StmtCtx - tableID := coll.PhysicalID - // TODO: If len(exprs) is bigger than 63, we could use bitset structure to replace the int64. - // This will simplify some code and speed up if we use this rather than a boolean slice. - if len(exprs) > 63 || (len(coll.Columns) == 0 && len(coll.Indices) == 0) { - ret = pseudoSelectivity(coll, exprs) - if sc.EnableOptimizerCETrace { - CETraceExpr(ctx, tableID, "Table Stats-Pseudo-Expression", expression.ComposeCNFCondition(ctx, exprs...), ret*float64(coll.RealtimeCount)) - } - return ret, nil, nil - } - - var nodes []*StatsNode - - var remainedExprStrs []string - remainedExprs := make([]expression.Expression, 0, len(exprs)) - - // Deal with the correlated column. - for i, expr := range exprs { - c := isColEqCorCol(expr) - if c == nil { - remainedExprs = append(remainedExprs, expr) - if sc.EnableOptimizerDebugTrace { - remainedExprStrs = append(remainedExprStrs, exprStrs[i]) - } - continue - } - - colHist := coll.Columns[c.UniqueID] - var sel float64 - if colHist == nil || colHist.IsInvalid(ctx, coll.Pseudo) { - sel = 1.0 / pseudoEqualRate - } else if colHist.Histogram.NDV > 0 { - sel = 1 / float64(colHist.Histogram.NDV) - } else { - sel = 1.0 / pseudoEqualRate - } - if sc.EnableOptimizerDebugTrace { - debugtrace.RecordAnyValuesWithNames(ctx, "Expression", expr.String(), "Selectivity", sel) - } - ret *= sel - } - - extractedCols := make([]*expression.Column, 0, len(coll.Columns)) - extractedCols = expression.ExtractColumnsFromExpressions(extractedCols, remainedExprs, nil) - colIDs := maps.Keys(coll.Columns) - slices.Sort(colIDs) - for _, id := range colIDs { - colStats := coll.Columns[id] - col := expression.ColInfo2Col(extractedCols, colStats.Info) - if col != nil { - maskCovered, ranges, _, err := getMaskAndRanges(ctx, remainedExprs, ranger.ColumnRangeType, nil, nil, col) - if err != nil { - return 0, nil, errors.Trace(err) - } - nodes = append(nodes, &StatsNode{Tp: ColType, ID: id, mask: maskCovered, Ranges: ranges, numCols: 1}) - if colStats.IsHandle { - nodes[len(nodes)-1].Tp = PkType - var cnt float64 - cnt, err = coll.GetRowCountByIntColumnRanges(ctx, id, ranges) - if err != nil { - return 0, nil, errors.Trace(err) - } - nodes[len(nodes)-1].Selectivity = cnt / float64(coll.RealtimeCount) - continue - } - cnt, err := coll.GetRowCountByColumnRanges(ctx, id, ranges) - if err != nil { - return 0, nil, errors.Trace(err) - } - nodes[len(nodes)-1].Selectivity = cnt / float64(coll.RealtimeCount) - } - } - id2Paths := make(map[int64]*planutil.AccessPath) - for _, path := range filledPaths { - // Index merge path and table path don't have index. - if path.Index == nil { - continue - } - id2Paths[path.Index.ID] = path - } - idxIDs := maps.Keys(coll.Indices) - slices.Sort(idxIDs) - for _, id := range idxIDs { - idxStats := coll.Indices[id] - idxCols := FindPrefixOfIndexByCol(extractedCols, coll.Idx2ColumnIDs[id], id2Paths[idxStats.ID]) - if len(idxCols) > 0 { - lengths := make([]int, 0, len(idxCols)) - for i := 0; i < len(idxCols) && i < len(idxStats.Info.Columns); i++ { - lengths = append(lengths, idxStats.Info.Columns[i].Length) - } - // If the found columns are more than the columns held by the index. We are appending the int pk to the tail of it. - // When storing index data to key-value store, we use (idx_col1, ...., idx_coln, handle_col) as its key. - if len(idxCols) > len(idxStats.Info.Columns) { - lengths = append(lengths, types.UnspecifiedLength) - } - maskCovered, ranges, partCover, err := getMaskAndRanges(ctx, remainedExprs, ranger.IndexRangeType, lengths, id2Paths[idxStats.ID], idxCols...) - if err != nil { - return 0, nil, errors.Trace(err) - } - cnt, err := coll.GetRowCountByIndexRanges(ctx, id, ranges) - if err != nil { - return 0, nil, errors.Trace(err) - } - selectivity := cnt / float64(coll.RealtimeCount) - nodes = append(nodes, &StatsNode{ - Tp: IndexType, - ID: id, - mask: maskCovered, - Ranges: ranges, - numCols: len(idxStats.Info.Columns), - Selectivity: selectivity, - partCover: partCover, - }) - } - } - usedSets := GetUsableSetsByGreedy(nodes) - // Initialize the mask with the full set. - mask := (int64(1) << uint(len(remainedExprs))) - 1 - // curExpr records covered expressions by now. It's for cardinality estimation tracing. - var curExpr []expression.Expression - - for _, set := range usedSets { - mask &^= set.mask - ret *= set.Selectivity - // If `partCover` is true, it means that the conditions are in DNF form, and only part - // of the DNF expressions are extracted as access conditions, so besides from the selectivity - // of the extracted access conditions, we multiply another selectionFactor for the residual - // conditions. - if set.partCover { - ret *= selectionFactor - } - if sc.EnableOptimizerCETrace { - // Tracing for the expression estimation results after applying this StatsNode. - for i := range remainedExprs { - if set.mask&(1< 0 { - curExpr = append(curExpr, remainedExprs[i]) - } - } - expr := expression.ComposeCNFCondition(ctx, curExpr...) - CETraceExpr(ctx, tableID, "Table Stats-Expression-CNF", expr, ret*float64(coll.RealtimeCount)) - } else if sc.EnableOptimizerDebugTrace { - var strs []string - for i := range remainedExprs { - if set.mask&(1< 0 { - strs = append(strs, remainedExprStrs[i]) - } - } - debugtrace.RecordAnyValuesWithNames(ctx, - "Expressions", strs, - "Selectivity", set.Selectivity, - "partial cover", set.partCover, - ) - } - } - - notCoveredConstants := make(map[int]*expression.Constant) - notCoveredDNF := make(map[int]*expression.ScalarFunction) - notCoveredStrMatch := make(map[int]*expression.ScalarFunction) - notCoveredNegateStrMatch := make(map[int]*expression.ScalarFunction) - notCoveredOtherExpr := make(map[int]expression.Expression) - if mask > 0 { - for i, expr := range remainedExprs { - if mask&(1< 0 { - minSelectivity := 1.0 - if len(notCoveredConstants) > 0 || len(notCoveredDNF) > 0 || len(notCoveredOtherExpr) > 0 { - minSelectivity = math.Min(minSelectivity, selectionFactor) - } - if len(notCoveredStrMatch) > 0 { - minSelectivity = math.Min(minSelectivity, ctx.GetSessionVars().GetStrMatchDefaultSelectivity()) - } - if len(notCoveredNegateStrMatch) > 0 { - minSelectivity = math.Min(minSelectivity, ctx.GetSessionVars().GetNegateStrMatchDefaultSelectivity()) - } - ret *= minSelectivity - if sc.EnableOptimizerDebugTrace { - debugtrace.RecordAnyValuesWithNames(ctx, "Default Selectivity", minSelectivity) - } - } - - if sc.EnableOptimizerCETrace { - // Tracing for the expression estimation results after applying the default selectivity. - totalExpr := expression.ComposeCNFCondition(ctx, remainedExprs...) - CETraceExpr(ctx, tableID, "Table Stats-Expression-CNF", totalExpr, ret*float64(coll.RealtimeCount)) - } - return ret, nodes, nil -} - -func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, rangeType ranger.RangeType, lengths []int, cachedPath *planutil.AccessPath, cols ...*expression.Column) (mask int64, ranges []*ranger.Range, partCover bool, err error) { - isDNF := false - var accessConds, remainedConds []expression.Expression - switch rangeType { - case ranger.ColumnRangeType: - accessConds = ranger.ExtractAccessConditionsForColumn(ctx, exprs, cols[0]) - ranges, accessConds, _, err = ranger.BuildColumnRange(accessConds, ctx, cols[0].RetType, types.UnspecifiedLength, ctx.GetSessionVars().RangeMaxSize) - case ranger.IndexRangeType: - if cachedPath != nil { - ranges, accessConds, remainedConds, isDNF = cachedPath.Ranges, cachedPath.AccessConds, cachedPath.TableFilters, cachedPath.IsDNFCond - break - } - var res *ranger.DetachRangeResult - res, err = ranger.DetachCondAndBuildRangeForIndex(ctx, exprs, cols, lengths, ctx.GetSessionVars().RangeMaxSize) - if err != nil { - return 0, nil, false, err - } - ranges, accessConds, remainedConds, isDNF = res.Ranges, res.AccessConds, res.RemainedConds, res.IsDNFCond - default: - panic("should never be here") - } - if err != nil { - return 0, nil, false, err - } - if isDNF && len(accessConds) > 0 { - mask |= 1 - return mask, ranges, len(remainedConds) > 0, nil - } - for i := range exprs { - for j := range accessConds { - if exprs[i].Equal(ctx, accessConds[j]) { - mask |= 1 << uint64(i) - break - } - } - } - return mask, ranges, false, nil -} - -// GetUsableSetsByGreedy will select the indices and pk used for calculate selectivity by greedy algorithm. -func GetUsableSetsByGreedy(nodes []*StatsNode) (newBlocks []*StatsNode) { - slices.SortFunc(nodes, func(i, j *StatsNode) int { - if r := compareType(i.Tp, j.Tp); r != 0 { - return r - } - return cmp.Compare(i.ID, j.ID) - }) - marked := make([]bool, len(nodes)) - mask := int64(math.MaxInt64) - for { - // Choose the index that covers most. - bestID, bestCount, bestTp, bestNumCols, bestMask, bestSel := -1, 0, ColType, 0, int64(0), float64(0) - for i, set := range nodes { - if marked[i] { - continue - } - curMask := set.mask & mask - if curMask != set.mask { - marked[i] = true - continue - } - bits := bits.OnesCount64(uint64(curMask)) - // This set cannot cover any thing, just skip it. - if bits == 0 { - marked[i] = true - continue - } - // We greedy select the stats info based on: - // (1): The stats type, always prefer the primary key or index. - // (2): The number of expression that it covers, the more the better. - // (3): The number of columns that it contains, the less the better. - // (4): The selectivity of the covered conditions, the less the better. - // The rationale behind is that lower selectivity tends to reflect more functional dependencies - // between columns. It's hard to decide the priority of this rule against rule 2 and 3, in order - // to avoid massive plan changes between tidb-server versions, I adopt this conservative strategy - // to impose this rule after rule 2 and 3. - if (bestTp == ColType && set.Tp != ColType) || - bestCount < bits || - (bestCount == bits && bestNumCols > set.numCols) || - (bestCount == bits && bestNumCols == set.numCols && bestSel > set.Selectivity) { - bestID, bestCount, bestTp, bestNumCols, bestMask, bestSel = i, bits, set.Tp, set.numCols, curMask, set.Selectivity - } - } - if bestCount == 0 { - break - } - - // Update the mask, remove the bit that nodes[bestID].mask has. - mask &^= bestMask - - newBlocks = append(newBlocks, nodes[bestID]) - marked[bestID] = true - } - return -} - -// FindPrefixOfIndexByCol will find columns in index by checking the unique id or the virtual expression. -// So it will return at once no matching column is found. -func FindPrefixOfIndexByCol(cols []*expression.Column, idxColIDs []int64, cachedPath *planutil.AccessPath) []*expression.Column { - if cachedPath != nil { - idxCols := cachedPath.IdxCols - retCols := make([]*expression.Column, 0, len(idxCols)) - idLoop: - for _, idCol := range idxCols { - for _, col := range cols { - if col.EqualByExprAndID(nil, idCol) { - retCols = append(retCols, col) - continue idLoop - } - } - // If no matching column is found, just return. - return retCols - } - return retCols - } - return expression.FindPrefixOfIndex(cols, idxColIDs) -} - -// CETraceExpr appends an expression and related information into CE trace -func CETraceExpr(sctx sessionctx.Context, tableID int64, tp string, expr expression.Expression, rowCount float64) { - exprStr, err := ExprToString(expr) - if err != nil { - logutil.BgLogger().Debug("Failed to trace CE of an expression", zap.String("category", "OptimizerTrace"), - zap.Any("expression", expr)) - return - } - rec := tracing.CETraceRecord{ - TableID: tableID, - Type: tp, - Expr: exprStr, - RowCount: uint64(rowCount), - } - sc := sctx.GetSessionVars().StmtCtx - sc.OptimizerCETrace = append(sc.OptimizerCETrace, &rec) -} - -// ExprToString prints an Expression into a string which can appear in a SQL. -// -// It might be too tricky because it makes use of TiDB allowing using internal function name in SQL. -// For example, you can write `eq`(a, 1), which is the same as a = 1. -// We should have implemented this by first implementing a method to turn an expression to an AST -// -// then call astNode.Restore(), like the Constant case here. But for convenience, we use this trick for now. -// -// It may be more appropriate to put this in expression package. But currently we only use it for CE trace, -// -// and it may not be general enough to handle all possible expressions. So we put it here for now. -func ExprToString(e expression.Expression) (string, error) { - switch expr := e.(type) { - case *expression.ScalarFunction: - var buffer bytes.Buffer - buffer.WriteString("`" + expr.FuncName.L + "`(") - switch expr.FuncName.L { - case ast.Cast: - for _, arg := range expr.GetArgs() { - argStr, err := ExprToString(arg) - if err != nil { - return "", err - } - buffer.WriteString(argStr) - buffer.WriteString(", ") - buffer.WriteString(expr.RetType.String()) - } - default: - for i, arg := range expr.GetArgs() { - argStr, err := ExprToString(arg) - if err != nil { - return "", err - } - buffer.WriteString(argStr) - if i+1 != len(expr.GetArgs()) { - buffer.WriteString(", ") - } - } - } - buffer.WriteString(")") - return buffer.String(), nil - case *expression.Column: - return expr.String(), nil - case *expression.CorrelatedColumn: - return "", errors.New("tracing for correlated columns not supported now") - case *expression.Constant: - value, err := expr.Eval(chunk.Row{}) - if err != nil { - return "", err - } - valueExpr := driver.ValueExpr{Datum: value} - var buffer bytes.Buffer - restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, &buffer) - err = valueExpr.Restore(restoreCtx) - if err != nil { - return "", err - } - return buffer.String(), nil - } - return "", errors.New("unexpected type of Expression") -} diff --git a/statistics/testdata/integration_suite_in.json b/statistics/testdata/integration_suite_in.json index befc8c7ea9fa3..17afe048b3419 100644 --- a/statistics/testdata/integration_suite_in.json +++ b/statistics/testdata/integration_suite_in.json @@ -16,142 +16,5 @@ "show stats_topn", "show stats_buckets" ] - }, - { - "name": "TestDefaultSelectivityForStrMatch", - "cases": [ - "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" - ] - }, - { - "name": "TestTopNAssistedEstimationWithNewCollation", - "cases": [ - "explain format = 'brief' select * from t where a like '%111%'", - "explain format = 'brief' select * from t where a not like '%111%'", - "explain format = 'brief' select * from t where a regexp '.*111.*'", - "explain format = 'brief' select * from t where a not regexp '.*111.*'", - "explain format = 'brief' select * from t where ifnull(a, '1111') like '%111%'", - "explain format = 'brief' select * from t where f like '%111%'", - "explain format = 'brief' select * from t where f not like '%111%'", - "explain format = 'brief' select * from t where f regexp '.*111.*'", - "explain format = 'brief' select * from t where f not regexp '.*111.*'", - "explain format = 'brief' select * from t where ifnull(f, '1111') like '%111%'", - "explain format = 'brief' select * from t where a like '%111%' and f rlike '.*111.*'", - "explain format = 'brief' select * from t where a like '%111%' or f like '%111%'", - "explain format = 'brief' select * from t where b like '%111%'", - "explain format = 'brief' select * from t where b not like '%111%'", - "explain format = 'brief' select * from t where b regexp '.*111.*'", - "explain format = 'brief' select * from t where b not regexp '.*111.*'", - "explain format = 'brief' select * from t where ifnull(b, '1111') like '%111%'", - "explain format = 'brief' select * from t where b like '%111%' and f like '%111%'", - "explain format = 'brief' select * from t where b like '%111%' or f like '%111%'", - "explain format = 'brief' select * from t where c like '%111%'", - "explain format = 'brief' select * from t where c not like '%111%'", - "explain format = 'brief' select * from t where ifnull(c, '1111') like '%111%'", - "explain format = 'brief' select * from t where d like '%111%'", - "explain format = 'brief' select * from t where d not like '%111%'", - "explain format = 'brief' select * from t where ifnull(c, '1111') like '%111%'", - "explain format = 'brief' select * from t where e like '%111%'", - "explain format = 'brief' select * from t where e not like '%111%'", - "explain format = 'brief' select * from t where ifnull(e, '1111') like '%111%'" - ] - }, - { - "name": "TestTopNAssistedEstimationWithoutNewCollation", - "cases": [ - "explain format = 'brief' select * from t where a like '%111%'", - "explain format = 'brief' select * from t where a not like '%111%'", - "explain format = 'brief' select * from t where a regexp '.*111.*'", - "explain format = 'brief' select * from t where a not regexp '.*111.*'", - "explain format = 'brief' select * from t where ifnull(a, '1111') like '%111%'", - "explain format = 'brief' select * from t where f like '%111%'", - "explain format = 'brief' select * from t where f not like '%111%'", - "explain format = 'brief' select * from t where f regexp '.*111.*'", - "explain format = 'brief' select * from t where f not regexp '.*111.*'", - "explain format = 'brief' select * from t where ifnull(f, '1111') like '%111%'", - "explain format = 'brief' select * from t where a like '%111%' and f rlike '.*111.*'", - "explain format = 'brief' select * from t where a like '%111%' or f like '%111%'", - "explain format = 'brief' select * from t where b like '%111%'", - "explain format = 'brief' select * from t where b not like '%111%'", - "explain format = 'brief' select * from t where b regexp '.*111.*'", - "explain format = 'brief' select * from t where b not regexp '.*111.*'", - "explain format = 'brief' select * from t where ifnull(b, '1111') like '%111%'", - "explain format = 'brief' select * from t where b like '%111%' and f like '%111%'", - "explain format = 'brief' select * from t where b like '%111%' or f like '%111%'", - "explain format = 'brief' select * from t where c like '%111%'", - "explain format = 'brief' select * from t where c not like '%111%'", - "explain format = 'brief' select * from t where ifnull(c, '1111') like '%111%'", - "explain format = 'brief' select * from t where d like '%111%'", - "explain format = 'brief' select * from t where d not like '%111%'", - "explain format = 'brief' select * from t where ifnull(c, '1111') like '%111%'", - "explain format = 'brief' select * from t where e like '%111%'", - "explain format = 'brief' select * from t where e not like '%111%'", - "explain format = 'brief' select * from t where ifnull(e, '1111') like '%111%'" - ] - }, - { - "name": "TestOrderingIdxSelectivityThreshold", - "cases": [ - "set @@tidb_opt_ordering_index_selectivity_threshold = 0", - "explain format = 'brief' select * from t where b >= 9950 order by c limit 1", - "explain format = 'brief' select * from t where b >= 9950 order by c desc limit 1", - "explain format = 'brief' select * from t where b >= 8999 order by c limit 1", - "explain format = 'brief' select * from t where b >= 9000 order by c limit 1", - "explain format = 'brief' select * from t where b >= 9001 order by c limit 1", - "explain format = 'brief' select * from t where a < 10001 order by c limit 1", - "explain format = 'brief' select * from t where a < 10000 order by c limit 1", - "explain format = 'brief' select * from t where a < 9999 order by c limit 1", - "explain format = 'brief' select * from t where b >= 0 and b <= 100 or c >= 0 and c <= 100 order by c limit 1", - "set @@tidb_opt_ordering_index_selectivity_threshold = 0.1", - "explain format = 'brief' select * from t where b >= 9950 order by c limit 1", - "explain format = 'brief' select * from t where b >= 9950 order by c desc limit 1", - "explain format = 'brief' select * from t where b >= 8999 order by c limit 1", - "explain format = 'brief' select * from t where b >= 9000 order by c limit 1", - "explain format = 'brief' select * from t where b >= 9001 order by c limit 1", - "explain format = 'brief' select * from t where a < 10001 order by c limit 1", - "explain format = 'brief' select * from t where a < 10000 order by c limit 1", - "explain format = 'brief' select * from t where a < 9999 order by c limit 1", - "explain format = 'brief' select * from t where b >= 0 and b <= 50 or c >= 0 and c <= 50 order by c limit 1", - "explain format = 'brief' select * from t where b >= 9950 and c >= 9950 order by c limit 1", - "explain format = 'brief' select * from t where b >= 9950 and c >= 9900 order by c limit 1" - ] } ] diff --git a/statistics/testdata/integration_suite_out.json b/statistics/testdata/integration_suite_out.json index 600a5187dc25e..e2706b96dd588 100644 --- a/statistics/testdata/integration_suite_out.json +++ b/statistics/testdata/integration_suite_out.json @@ -47,980 +47,5 @@ "test t idx 1 1 2 1 4 4 0" ] ] - }, - { - "Name": "TestDefaultSelectivityForStrMatch", - "Cases": [ - { - "SQL": "set @@tidb_default_string_match_selectivity = 0.8", - "Result": null - }, - { - "SQL": "explain format = 'brief' select * from t where a + 10 > 100", - "Result": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b like '%test%'", - "Result": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tikv] like(test.t.b, \"%test%\", 92)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b not like '%test%'", - "Result": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tikv] not(like(test.t.b, \"%test%\", 92))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b regexp '.*test.*'", - "Result": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tikv] regexp(test.t.b, \".*test.*\")", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b not regexp '.*test.*'", - "Result": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tikv] not(istrue_with_null(regexp(test.t.b, \".*test.*\")))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100", - "Result": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100), like(test.t.b, \"%test%\", 92)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100", - "Result": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100), not(like(test.t.b, \"%test%\", 92))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100", - "Result": [ - "TableReader 9600.00 root data:Selection", - "└─Selection 9600.00 cop[tikv] or(like(test.t.b, \"%test%\", 92), gt(plus(test.t.a, 10), 100))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "set @@tidb_default_string_match_selectivity = 0.9", - "Result": null - }, - { - "SQL": "explain format = 'brief' select * from t where a + 10 > 100", - "Result": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b like '%test%'", - "Result": [ - "TableReader 9000.00 root data:Selection", - "└─Selection 9000.00 cop[tikv] like(test.t.b, \"%test%\", 92)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b not like '%test%'", - "Result": [ - "TableReader 1000.00 root data:Selection", - "└─Selection 1000.00 cop[tikv] not(like(test.t.b, \"%test%\", 92))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b regexp '.*test.*'", - "Result": [ - "TableReader 9000.00 root data:Selection", - "└─Selection 9000.00 cop[tikv] regexp(test.t.b, \".*test.*\")", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b not regexp '.*test.*'", - "Result": [ - "TableReader 1000.00 root data:Selection", - "└─Selection 1000.00 cop[tikv] not(istrue_with_null(regexp(test.t.b, \".*test.*\")))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100", - "Result": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100), like(test.t.b, \"%test%\", 92)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100", - "Result": [ - "TableReader 1000.00 root data:Selection", - "└─Selection 1000.00 cop[tikv] gt(plus(test.t.a, 10), 100), not(like(test.t.b, \"%test%\", 92))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100", - "Result": [ - "TableReader 9800.00 root data:Selection", - "└─Selection 9800.00 cop[tikv] or(like(test.t.b, \"%test%\", 92), gt(plus(test.t.a, 10), 100))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "set @@tidb_default_string_match_selectivity = 0.1", - "Result": null - }, - { - "SQL": "explain format = 'brief' select * from t where a + 10 > 100", - "Result": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b like '%test%'", - "Result": [ - "TableReader 1000.00 root data:Selection", - "└─Selection 1000.00 cop[tikv] like(test.t.b, \"%test%\", 92)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b like '%test%' is true", - "Result": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tikv] istrue(like(test.t.b, \"%test%\", 92))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b not like '%test%'", - "Result": [ - "TableReader 9000.00 root data:Selection", - "└─Selection 9000.00 cop[tikv] not(like(test.t.b, \"%test%\", 92))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b regexp '.*test.*'", - "Result": [ - "TableReader 1000.00 root data:Selection", - "└─Selection 1000.00 cop[tikv] regexp(test.t.b, \".*test.*\")", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b rlike '.*test.*'", - "Result": [ - "TableReader 1000.00 root data:Selection", - "└─Selection 1000.00 cop[tikv] regexp(test.t.b, \".*test.*\")", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b not rlike '.*test.*'", - "Result": [ - "TableReader 9000.00 root data:Selection", - "└─Selection 9000.00 cop[tikv] not(istrue_with_null(regexp(test.t.b, \".*test.*\")))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b not regexp '.*test.*'", - "Result": [ - "TableReader 9000.00 root data:Selection", - "└─Selection 9000.00 cop[tikv] not(istrue_with_null(regexp(test.t.b, \".*test.*\")))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100", - "Result": [ - "TableReader 1000.00 root data:Selection", - "└─Selection 1000.00 cop[tikv] gt(plus(test.t.a, 10), 100), like(test.t.b, \"%test%\", 92)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100", - "Result": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100), not(like(test.t.b, \"%test%\", 92))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100", - "Result": [ - "TableReader 8200.00 root data:Selection", - "└─Selection 8200.00 cop[tikv] or(like(test.t.b, \"%test%\", 92), gt(plus(test.t.a, 10), 100))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "set @@tidb_default_string_match_selectivity = 0", - "Result": null - }, - { - "SQL": "explain format = 'brief' select * from t where a + 10 > 100", - "Result": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b like '%test%'", - "Result": [ - "TableReader 1000.00 root data:Selection", - "└─Selection 1000.00 cop[tikv] like(test.t.b, \"%test%\", 92)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b not like '%test%'", - "Result": [ - "TableReader 9000.00 root data:Selection", - "└─Selection 9000.00 cop[tikv] not(like(test.t.b, \"%test%\", 92))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b regexp '.*test.*'", - "Result": [ - "TableReader 1000.00 root data:Selection", - "└─Selection 1000.00 cop[tikv] regexp(test.t.b, \".*test.*\")", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b not regexp '.*test.*'", - "Result": [ - "TableReader 9000.00 root data:Selection", - "└─Selection 9000.00 cop[tikv] not(istrue_with_null(regexp(test.t.b, \".*test.*\")))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b like '%test%' and a + 10 > 100", - "Result": [ - "TableReader 1000.00 root data:Selection", - "└─Selection 1000.00 cop[tikv] gt(plus(test.t.a, 10), 100), like(test.t.b, \"%test%\", 92)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b not like '%test%' and a + 10 > 100", - "Result": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tikv] gt(plus(test.t.a, 10), 100), not(like(test.t.b, \"%test%\", 92))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b like '%test%' or a + 10 > 100", - "Result": [ - "TableReader 8200.00 root data:Selection", - "└─Selection 8200.00 cop[tikv] or(like(test.t.b, \"%test%\", 92), gt(plus(test.t.a, 10), 100))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "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": "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": "TestOrderingIdxSelectivityThreshold", - "Cases": [ - { - "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": "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" - ] - } - ] } ] diff --git a/statistics/testdata/stats_suite_in.json b/statistics/testdata/stats_suite_in.json deleted file mode 100644 index bf53c6726c974..0000000000000 --- a/statistics/testdata/stats_suite_in.json +++ /dev/null @@ -1,299 +0,0 @@ -[ - { - "name": "TestUniqCompEqualEst", - "cases": [ - "explain select * from t where a = 1 and b = 5 and 1 = 1" - ] - }, - { - "name": "TestStatsVer2", - "cases": [ - "show stats_topn", - "show stats_buckets", - "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" - ] - }, - { - "name": "TestTopNOutOfHist", - "cases": [ - "show stats_topn", - "show stats_buckets", - "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" - ] - }, - { - "name": "TestColumnIndexNullEstimation", - "cases": [ - "explain select b from t where b is null", - "explain select b from t where b is not null", - "explain select b from t where b is null or b > 3", - "explain select b from t use index(idx_b)", - "explain select b from t where b < 4", - "explain select * from t where a is null", - "explain select * from t where a is not null", - "explain select * from t where a is null or a > 3", - "explain select * from t", - "explain select * from t where a < 4" - ] - }, - { - "name": "TestDiscreteDistribution", - "cases": [ - "explain select * from t where a = 'tw' and b < 0" - ] - }, - { - "name": "TestPrimaryKeySelectivity", - "cases": [ - [ - "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" - ] - ] - }, - { - "name": "TestSelectCombinedLowBound", - "cases": [ - "explain select * from t where kid = 1" - ] - }, - { - "name": "TestCollationColumnEstimate", - "cases": [ - "show stats_topn", - "explain select * from t where a='aÄa'", - "explain select * from t where a > 'aÄa'" - ] - }, - { - "name": "TestDNFCondSelectivity", - "cases": [ - "select * from t where b > 7 or c < 4", - "select * from t where d < 5 or b > 6", - "select * from t where a > 8 or d < 4 or c > 7 or b < 5", - "select * from t where a < 8 and (b > 10 or c < 3 or b > 4) and a > 2" - ] - }, - { - "Name": "TestSmallRangeEstimation", - "Cases": [ - { - "Start": 5, - "End": 5 - }, - { - "Start": 5, - "End": 6 - }, - { - "Start": 5, - "End": 10 - }, - { - "Start": 5, - "End": 15 - }, - { - "Start": 10, - "End": 15 - }, - { - "Start": 5, - "End": 25 - }, - { - "Start": 25, - "End": 25 - } - ] - }, - { - "Name": "TestOutOfRangeEstimation", - "Cases": [ - { - "Start": 800, - "End": 900 - }, - { - "Start": 900, - "End": 950 - }, - { - "Start": 950, - "End": 1000 - }, - { - "Start": 1000, - "End": 1050 - }, - { - "Start": 1050, - "End": 1100 - }, - { - "Start": 1150, - "End": 1200 - }, - { - "Start": 1200, - "End": 1300 - }, - { - "Start": 1300, - "End": 1400 - }, - { - "Start": 1400, - "End": 1500 - }, - { - "Start": 1500, - "End": 1600 - }, - { - "Start": 300, - "End": 899 - }, - { - "Start": 800, - "End": 1000 - }, - { - "Start": 900, - "End": 1500 - }, - { - "Start": 300, - "End": 1500 - }, - { - "Start": 200, - "End": 300 - }, - { - "Start": 100, - "End": 200 - }, - { - "Start": 200, - "End": 400 - }, - { - "Start": 200, - "End": 1000 - }, - { - "Start": 0, - "End": 100 - }, - { - "Start": -100, - "End": 100 - }, - { - "Start": -100, - "End": 0 - } - ] - }, - { - "name": "TestOutOfRangeEstimationAfterDelete", - "cases": [ - "explain format = 'brief' select * from t where a <= 300", - "explain format = 'brief' select * from t where a < 300", - "explain format = 'brief' select * from t where a <= 500", - "explain format = 'brief' select * from t where a >= 300 and a <= 900", - "explain format = 'brief' select * from t where a >= 900", - "explain format = 'brief' select * from t where a > 900", - "explain format = 'brief' select * from t where a >= 300", - "explain format = 'brief' select * from t where a <= 900", - "explain format = 'brief' select * from t where a > 800 and a < 1000", - "explain format = 'brief' select * from t where a > 900 and a < 1000", - "explain format = 'brief' select * from t where a > 900 and a < 1100", - "explain format = 'brief' select * from t where a > 200 and a < 300", - "explain format = 'brief' select * from t where a > 100 and a < 300" - ] - }, - { - "name": "TestGlobalStatsOutOfRangeEstimationAfterDelete", - "cases": [ - "explain format = 'brief' select * from t where a <= 300", - "explain format = 'brief' select * from t where a < 300", - "explain format = 'brief' select * from t where a <= 500", - "explain format = 'brief' select * from t where a >= 300 and a <= 900", - "explain format = 'brief' select * from t where a >= 900", - "explain format = 'brief' select * from t where a > 900", - "explain format = 'brief' select * from t where a >= 300", - "explain format = 'brief' select * from t where a <= 900", - "explain format = 'brief' select * from t where a > 800 and a < 1000", - "explain format = 'brief' select * from t where a > 900 and a < 1000", - "explain format = 'brief' select * from t where a > 900 and a < 1100", - "explain format = 'brief' select * from t where a > 200 and a < 300", - "explain format = 'brief' select * from t where a > 100 and a < 300" - ] - } -] diff --git a/statistics/testdata/stats_suite_out.json b/statistics/testdata/stats_suite_out.json deleted file mode 100644 index fea01ef77bee1..0000000000000 --- a/statistics/testdata/stats_suite_out.json +++ /dev/null @@ -1,981 +0,0 @@ -[ - { - "Name": "TestUniqCompEqualEst", - "Cases": [ - [ - "Point_Get_5 1.00 root table:t, clustered index:PRIMARY(a, b) " - ] - ] - }, - { - "Name": "TestStatsVer2", - "Cases": [ - [ - "test tint a 0 1 1", - "test tint a 0 2 1", - "test tint b 0 1 1", - "test tint b 0 2 1", - "test tint c 0 1 1", - "test tint c 0 2 1", - "test tint singular 1 1 1", - "test tint singular 1 2 1", - "test tint multi 1 (1, 1) 1", - "test tint multi 1 (2, 2) 1", - "test tdouble a 0 1 1", - "test tdouble a 0 2 1", - "test tdouble b 0 1 1", - "test tdouble b 0 2 1", - "test tdouble c 0 1 1", - "test tdouble c 0 2 1", - "test tdouble singular 1 1 1", - "test tdouble singular 1 2 1", - "test tdouble multi 1 (1, 1) 1", - "test tdouble multi 1 (2, 2) 1", - "test tdecimal a 0 1.00000000000000000000 1", - "test tdecimal a 0 2.00000000000000000000 1", - "test tdecimal b 0 1.00000000000000000000 1", - "test tdecimal b 0 2.00000000000000000000 1", - "test tdecimal c 0 1.00000000000000000000 1", - "test tdecimal c 0 2.00000000000000000000 1", - "test tdecimal singular 1 1.00000000000000000000 1", - "test tdecimal singular 1 2.00000000000000000000 1", - "test tdecimal multi 1 (1.00000000000000000000, 1.00000000000000000000) 1", - "test tdecimal multi 1 (2.00000000000000000000, 2.00000000000000000000) 1", - "test tstring a 0 1 1", - "test tstring a 0 2 1", - "test tstring b 0 1 1", - "test tstring b 0 2 1", - "test tstring c 0 1 1", - "test tstring c 0 2 1", - "test tstring singular 1 1 1", - "test tstring singular 1 2 1", - "test tstring multi 1 (1, 1) 1", - "test tstring multi 1 (2, 2) 1", - "test tdatetime a 0 2001-01-01 00:00:00 1", - "test tdatetime a 0 2001-01-02 00:00:00 1", - "test tdatetime b 0 2001-01-01 00:00:00 1", - "test tdatetime b 0 2001-01-02 00:00:00 1", - "test tdatetime c 0 2001-01-01 00:00:00 1", - "test tdatetime c 0 2001-01-02 00:00:00 1", - "test tdatetime singular 1 2001-01-01 00:00:00 1", - "test tdatetime singular 1 2001-01-02 00:00:00 1", - "test tdatetime multi 1 (2001-01-01 00:00:00, 2001-01-01 00:00:00) 1", - "test tdatetime multi 1 (2001-01-02 00:00:00, 2001-01-02 00:00:00) 1", - "test tprefix a 0 111 1", - "test tprefix a 0 222 1", - "test tprefix b 0 111 1", - "test tprefix b 0 222 1", - "test tprefix prefixa 1 11 1", - "test tprefix prefixa 1 22 1", - "test ct1 a 0 1 1", - "test ct1 a 0 2 1", - "test ct1 pk 0 1 1", - "test ct1 pk 0 2 1", - "test ct1 PRIMARY 1 1 1", - "test ct1 PRIMARY 1 2 1", - "test ct2 a 0 1 1", - "test ct2 a 0 2 1", - "test ct2 b 0 1 1", - "test ct2 b 0 2 1", - "test ct2 c 0 1 1", - "test ct2 c 0 2 1", - "test ct2 PRIMARY 1 (1, 1) 1", - "test ct2 PRIMARY 1 (2, 2) 1" - ], - [ - "test tint a 0 0 3 1 3 5 0", - "test tint a 0 1 6 1 6 8 0", - "test tint b 0 0 3 1 3 5 0", - "test tint b 0 1 6 1 6 8 0", - "test tint c 0 0 3 1 3 5 0", - "test tint c 0 1 6 1 6 8 0", - "test tint singular 1 0 3 1 3 5 0", - "test tint singular 1 1 6 1 6 8 0", - "test tint multi 1 0 3 1 (3, 3) (5, 5) 0", - "test tint multi 1 1 6 1 (6, 6) (8, 8) 0", - "test tdouble a 0 0 3 1 3 5 0", - "test tdouble a 0 1 6 1 6 8 0", - "test tdouble b 0 0 3 1 3 5 0", - "test tdouble b 0 1 6 1 6 8 0", - "test tdouble c 0 0 3 1 3 5 0", - "test tdouble c 0 1 6 1 6 8 0", - "test tdouble singular 1 0 3 1 3 5 0", - "test tdouble singular 1 1 6 1 6 8 0", - "test tdouble multi 1 0 3 1 (3, 3) (5, 5) 0", - "test tdouble multi 1 1 6 1 (6, 6) (8, 8) 0", - "test tdecimal a 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0", - "test tdecimal a 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0", - "test tdecimal b 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0", - "test tdecimal b 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0", - "test tdecimal c 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0", - "test tdecimal c 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0", - "test tdecimal singular 1 0 3 1 3.00000000000000000000 5.00000000000000000000 0", - "test tdecimal singular 1 1 6 1 6.00000000000000000000 8.00000000000000000000 0", - "test tdecimal multi 1 0 3 1 (3.00000000000000000000, 3.00000000000000000000) (5.00000000000000000000, 5.00000000000000000000) 0", - "test tdecimal multi 1 1 6 1 (6.00000000000000000000, 6.00000000000000000000) (8.00000000000000000000, 8.00000000000000000000) 0", - "test tstring a 0 0 3 1 3 5 0", - "test tstring a 0 1 6 1 6 8 0", - "test tstring b 0 0 3 1 3 5 0", - "test tstring b 0 1 6 1 6 8 0", - "test tstring c 0 0 3 1 3 5 0", - "test tstring c 0 1 6 1 6 8 0", - "test tstring singular 1 0 3 1 3 5 0", - "test tstring singular 1 1 6 1 6 8 0", - "test tstring multi 1 0 3 1 (3, 3) (5, 5) 0", - "test tstring multi 1 1 6 1 (6, 6) (8, 8) 0", - "test tdatetime a 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0", - "test tdatetime a 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0", - "test tdatetime b 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0", - "test tdatetime b 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0", - "test tdatetime c 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0", - "test tdatetime c 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0", - "test tdatetime singular 1 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0", - "test tdatetime singular 1 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0", - "test 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", - "test 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", - "test tprefix a 0 0 2 1 333 444 0", - "test tprefix a 0 1 4 1 555 666 0", - "test tprefix b 0 0 2 1 333 444 0", - "test tprefix b 0 1 4 1 555 666 0", - "test tprefix prefixa 1 0 2 1 33 44 0", - "test tprefix prefixa 1 1 4 1 55 66 0", - "test ct1 a 0 0 3 1 3 5 0", - "test ct1 a 0 1 6 1 6 8 0", - "test ct1 pk 0 0 3 1 3 5 0", - "test ct1 pk 0 1 6 1 6 8 0", - "test ct1 PRIMARY 1 0 3 1 3 5 0", - "test ct1 PRIMARY 1 1 6 1 6 8 0", - "test ct2 a 0 0 3 1 3 5 0", - "test ct2 a 0 1 6 1 6 8 0", - "test ct2 b 0 0 3 1 3 5 0", - "test ct2 b 0 1 6 1 6 8 0", - "test ct2 c 0 0 3 1 3 5 0", - "test ct2 c 0 1 6 1 6 8 0", - "test ct2 PRIMARY 1 0 3 1 (3, 3) (5, 5) 0", - "test ct2 PRIMARY 1 1 6 1 (6, 6) (8, 8) 0" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tint.a, 1)", - " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tint.a, 4)", - " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tint.a, 8)", - " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.a, 1)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.a, 4)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.a, 8)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.a, 1)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.a, 4)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.a, 8)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tstring.a, \"1\")", - " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tstring.a, \"4\")", - " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tstring.a, \"8\")", - " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.a, 2001-01-01 00:00:00.000000)", - " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.a, 2001-01-02 00:00:00.000000)", - " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.a, 2001-01-04 00:00:00.000000)", - " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tprefix.a, \"111\")", - " └─TableFullScan_5 6.00 cop[tikv] table:tprefix keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tprefix.a, \"444\")", - " └─TableFullScan_5 6.00 cop[tikv] table:tprefix keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tprefix.a, \"888\")", - " └─TableFullScan_5 6.00 cop[tikv] table:tprefix keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tint.b, 1), eq(test.tint.c, 1)", - " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tint.b, 4), eq(test.tint.c, 4)", - " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tint.b, 8), eq(test.tint.c, 8)", - " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 1), eq(test.tdouble.c, 1)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 4), eq(test.tdouble.c, 4)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 8), eq(test.tdouble.c, 8)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 1), eq(test.tdecimal.c, 1)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 4), eq(test.tdecimal.c, 4)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 8), eq(test.tdecimal.c, 8)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tstring.b, \"1\"), eq(test.tstring.c, \"1\")", - " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tstring.b, \"4\"), eq(test.tstring.c, \"4\")", - " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tstring.b, \"8\"), eq(test.tstring.c, \"8\")", - " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.b, 2001-01-01 00:00:00.000000), eq(test.tdatetime.c, 2001-01-01 00:00:00.000000)", - " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.b, 2001-01-02 00:00:00.000000), eq(test.tdatetime.c, 2001-01-02 00:00:00.000000)", - " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.b, 2001-01-04 00:00:00.000000), eq(test.tdatetime.c, 2001-01-04 00:00:00.000000)", - " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tint.b, 1)", - " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tint.b, 4)", - " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tint.b, 8)", - " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 1)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 4)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdouble.b, 8)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdouble keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 1)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 4)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdecimal.b, 8)", - " └─TableFullScan_5 8.00 cop[tikv] table:tdecimal keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tstring.b, \"1\")", - " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tstring.b, \"4\")", - " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tstring.b, \"8\")", - " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.b, 2001-01-01 00:00:00.000000)", - " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.b, 2001-01-02 00:00:00.000000)", - " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tdatetime.b, 2001-01-04 00:00:00.000000)", - " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" - ], - [ - "TableReader_6 5.00 root data:TableRangeScan_5", - "└─TableRangeScan_5 5.00 cop[tikv] table:ct1 range:[\"1\",\"4\"], keep order:false" - ], - [ - "TableReader_6 3.75 root data:TableRangeScan_5", - "└─TableRangeScan_5 3.75 cop[tikv] table:ct1 range:[\"4\",\"6\"], keep order:false" - ], - [ - "TableReader_6 3.00 root data:TableRangeScan_5", - "└─TableRangeScan_5 3.00 cop[tikv] table:ct1 range:[\"6\",\"8\"], keep order:false" - ], - [ - "TableReader_6 1.00 root data:TableRangeScan_5", - "└─TableRangeScan_5 1.00 cop[tikv] table:ct2 range:[1 1,1 8], keep order:false" - ], - [ - "TableReader_6 1.00 root data:TableRangeScan_5", - "└─TableRangeScan_5 1.00 cop[tikv] table:ct2 range:[4 1,4 8], keep order:false" - ], - [ - "TableReader_6 1.00 root data:TableRangeScan_5", - "└─TableRangeScan_5 1.00 cop[tikv] table:ct2 range:[8 1,8 8], keep order:false" - ] - ] - }, - { - "Name": "TestTopNOutOfHist", - "Cases": [ - [ - "test topn_before_hist a 0 1 4", - "test topn_before_hist a 0 3 2", - "test topn_before_hist idx 1 1 4", - "test topn_before_hist idx 1 3 2", - "test topn_after_hist a 0 2 2", - "test topn_after_hist a 0 7 4", - "test topn_after_hist idx 1 2 2", - "test topn_after_hist idx 1 7 4", - "test topn_before_hist_no_index a 0 1 4", - "test topn_before_hist_no_index a 0 3 2", - "test topn_after_hist_no_index a 0 2 2", - "test topn_after_hist_no_index a 0 7 4" - ], - [ - "test topn_before_hist a 0 0 2 1 4 5 0", - "test topn_before_hist a 0 1 3 1 6 6 0", - "test topn_before_hist idx 1 0 2 1 4 5 0", - "test topn_before_hist idx 1 1 3 1 6 6 0", - "test topn_after_hist a 0 0 2 1 3 4 0", - "test topn_after_hist a 0 1 3 1 5 5 0", - "test topn_after_hist idx 1 0 2 1 3 4 0", - "test topn_after_hist idx 1 1 3 1 5 5 0", - "test topn_before_hist_no_index a 0 0 2 1 4 5 0", - "test topn_before_hist_no_index a 0 1 3 1 6 6 0", - "test topn_after_hist_no_index a 0 0 2 1 3 4 0", - "test topn_after_hist_no_index a 0 1 3 1 5 5 0" - ], - [ - "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" - ], - [ - "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" - ], - [ - "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" - ], - [ - "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" - ], - [ - "TableReader_7 4.00 root data:Selection_6", - "└─Selection_6 4.00 cop[tikv] eq(test.topn_after_hist_no_index.a, 7)", - " └─TableFullScan_5 9.00 cop[tikv] table:topn_after_hist_no_index keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.topn_after_hist_no_index.a, 6)", - " └─TableFullScan_5 9.00 cop[tikv] table:topn_after_hist_no_index keep order:false" - ], - [ - "TableReader_7 4.00 root data:Selection_6", - "└─Selection_6 4.00 cop[tikv] eq(test.topn_before_hist_no_index.a, 1)", - " └─TableFullScan_5 9.00 cop[tikv] table:topn_before_hist_no_index keep order:false" - ], - [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.topn_before_hist_no_index.a, 2)", - " └─TableFullScan_5 9.00 cop[tikv] table:topn_before_hist_no_index 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": "TestDiscreteDistribution", - "Cases": [ - [ - "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" - ] - ] - }, - { - "Name": "TestPrimaryKeySelectivity", - "Cases": [ - [ - "TableReader_7 3333.33 root data:Selection_6", - "└─Selection_6 3333.33 cop[tikv] gt(test.t.a, \"t\")", - " └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ], - [ - "TableReader_6 3333.33 root data:TableRangeScan_5", - "└─TableRangeScan_5 3333.33 cop[tikv] table:t range:(1,+inf], keep order:false, stats:pseudo" - ] - ] - }, - { - "Name": "TestSelectCombinedLowBound", - "Cases": [ - [ - "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" - ] - ] - }, - { - "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": "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": "TestOutOfRangeEstimation", - "Cases": [ - { - "Start": 800, - "End": 900, - "Count": 752.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": 4498.5 - }, - { - "Start": 800, - "End": 1000, - "Count": 1201.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": 1211.5288209899081 - }, - { - "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.13 root data:Selection", - "└─Selection 793.13 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.12 root data:Selection", - "└─Selection 458.12 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.49 root data:Selection", - "└─Selection 832.49 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 458.12 root data:Selection", - "└─Selection 458.12 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 832.49 root data:Selection", - "└─Selection 832.49 cop[tikv] gt(test.t.a, 100), lt(test.t.a, 300)", - " └─TableFullScan 2000.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" - ] - } - ] - } -] diff --git a/statistics/testdata/trace_suite_out.json b/statistics/testdata/trace_suite_out.json index 4a1070905ddd8..5f3a0500e9c8e 100644 --- a/statistics/testdata/trace_suite_out.json +++ b/statistics/testdata/trace_suite_out.json @@ -161,7 +161,7 @@ { "ResultForV1": [ { - "github.com/pingcap/tidb/statistics.(*HistColl).Selectivity": [ + "github.com/pingcap/tidb/planner/cardinality.Selectivity": [ { "Input Expressions": [ "eq(test.t.a, 100)", @@ -718,7 +718,7 @@ ], "ResultForV2": [ { - "github.com/pingcap/tidb/statistics.(*HistColl).Selectivity": [ + "github.com/pingcap/tidb/planner/cardinality.Selectivity": [ { "Input Expressions": [ "eq(test.t.a, 100)", @@ -1089,7 +1089,7 @@ { "ResultForV1": [ { - "github.com/pingcap/tidb/statistics.(*HistColl).Selectivity": [ + "github.com/pingcap/tidb/planner/cardinality.Selectivity": [ { "Input Expressions": [ "lt(test.t.a, -1500)", @@ -2080,7 +2080,7 @@ ], "ResultForV2": [ { - "github.com/pingcap/tidb/statistics.(*HistColl).Selectivity": [ + "github.com/pingcap/tidb/planner/cardinality.Selectivity": [ { "Input Expressions": [ "lt(test.t.a, -1500)", diff --git a/statistics/trace_test.go b/statistics/trace_test.go index 4022a448e50f8..89b761adae4e1 100644 --- a/statistics/trace_test.go +++ b/statistics/trace_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/planner/cardinality" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" @@ -248,7 +249,7 @@ func TestTraceDebugSelectivity(t *testing.T) { for i, sql := range in { stmtCtx.OptimizerDebugTrace = nil histColl := statsTbl.GenerateHistCollFromColumnInfo(tblInfos[i], dsSchemaCols[i]) - _, _, err = histColl.Selectivity(sctx, selConditions[i], nil) + _, _, err = cardinality.Selectivity(sctx, histColl, selConditions[i], nil) require.NoError(t, err, sql, "For ver2") traceInfo := stmtCtx.OptimizerDebugTrace buf.Reset() @@ -272,7 +273,7 @@ func TestTraceDebugSelectivity(t *testing.T) { for i, sql := range in { stmtCtx.OptimizerDebugTrace = nil histColl := statsTbl.GenerateHistCollFromColumnInfo(tblInfos[i], dsSchemaCols[i]) - _, _, err = histColl.Selectivity(sctx, selConditions[i], nil) + _, _, err = cardinality.Selectivity(sctx, histColl, selConditions[i], nil) require.NoError(t, err, sql, "For ver1") traceInfo := stmtCtx.OptimizerDebugTrace buf.Reset()