From fe6d41676fe044fca2469f420423d82da1361a91 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Fri, 8 Sep 2023 18:10:36 +0800 Subject: [PATCH 1/9] lightning/external: add statistic for multiple data files (#46614) ref pingcap/tidb#45719 --- br/pkg/lightning/backend/external/BUILD.bazel | 2 +- br/pkg/lightning/backend/external/util.go | 48 +++++++ .../lightning/backend/external/util_test.go | 31 +++++ br/pkg/lightning/backend/external/writer.go | 110 +++++++++++++--- .../lightning/backend/external/writer_test.go | 119 ++++++++++++++++++ 5 files changed, 290 insertions(+), 20 deletions(-) diff --git a/br/pkg/lightning/backend/external/BUILD.bazel b/br/pkg/lightning/backend/external/BUILD.bazel index a01c8e96640b7..7776bc9ddd632 100644 --- a/br/pkg/lightning/backend/external/BUILD.bazel +++ b/br/pkg/lightning/backend/external/BUILD.bazel @@ -53,7 +53,7 @@ go_test( ], embed = [":external"], flaky = True, - shard_count = 28, + shard_count = 31, deps = [ "//br/pkg/lightning/backend/kv", "//br/pkg/lightning/common", diff --git a/br/pkg/lightning/backend/external/util.go b/br/pkg/lightning/backend/external/util.go index ca3deb5d52056..ee76b8b800ba3 100644 --- a/br/pkg/lightning/backend/external/util.go +++ b/br/pkg/lightning/backend/external/util.go @@ -19,6 +19,7 @@ import ( "context" "fmt" "path/filepath" + "slices" "sort" "strings" @@ -199,3 +200,50 @@ func MockExternalEngineWithWriter( } return GetAllFileNames(ctx, storage, subDir) } + +// EndpointTp is the type of Endpoint.Key. +type EndpointTp int + +const ( + // ExclusiveEnd represents "..., Endpoint.Key)". + ExclusiveEnd EndpointTp = iota + // InclusiveStart represents "[Endpoint.Key, ...". + InclusiveStart + // InclusiveEnd represents "..., Endpoint.Key]". + InclusiveEnd +) + +// Endpoint represents an endpoint of an interval which can be used by GetMaxOverlapping. +type Endpoint struct { + Key []byte + Tp EndpointTp + Weight uint64 // all EndpointTp use positive weight +} + +// GetMaxOverlapping returns the maximum overlapping weight treating given +// `points` as endpoints of intervals. `points` are not required to be sorted, +// and will be sorted in-place in this function. +func GetMaxOverlapping(points []Endpoint) int { + slices.SortFunc(points, func(i, j Endpoint) int { + if cmp := bytes.Compare(i.Key, j.Key); cmp != 0 { + return cmp + } + return int(i.Tp) - int(j.Tp) + }) + var maxWeight uint64 + var curWeight uint64 + for _, p := range points { + switch p.Tp { + case InclusiveStart: + curWeight += p.Weight + case ExclusiveEnd: + curWeight -= p.Weight + case InclusiveEnd: + curWeight -= p.Weight + } + if curWeight > maxWeight { + maxWeight = curWeight + } + } + return int(maxWeight) +} diff --git a/br/pkg/lightning/backend/external/util_test.go b/br/pkg/lightning/backend/external/util_test.go index 379ccd7060019..7c6678d9d6054 100644 --- a/br/pkg/lightning/backend/external/util_test.go +++ b/br/pkg/lightning/backend/external/util_test.go @@ -209,3 +209,34 @@ func TestCleanUpFiles(t *testing.T) { require.Equal(t, []string(nil), statFiles) require.Equal(t, []string(nil), dataFiles) } + +func TestGetMaxOverlapping(t *testing.T) { + // [1, 3), [2, 4) + points := []Endpoint{ + {Key: []byte{1}, Tp: InclusiveStart, Weight: 1}, + {Key: []byte{3}, Tp: ExclusiveEnd, Weight: 1}, + {Key: []byte{2}, Tp: InclusiveStart, Weight: 1}, + {Key: []byte{4}, Tp: ExclusiveEnd, Weight: 1}, + } + require.Equal(t, 2, GetMaxOverlapping(points)) + // [1, 3), [2, 4), [3, 5) + points = []Endpoint{ + {Key: []byte{1}, Tp: InclusiveStart, Weight: 1}, + {Key: []byte{3}, Tp: ExclusiveEnd, Weight: 1}, + {Key: []byte{2}, Tp: InclusiveStart, Weight: 1}, + {Key: []byte{4}, Tp: ExclusiveEnd, Weight: 1}, + {Key: []byte{3}, Tp: InclusiveStart, Weight: 1}, + {Key: []byte{5}, Tp: ExclusiveEnd, Weight: 1}, + } + require.Equal(t, 2, GetMaxOverlapping(points)) + // [1, 3], [2, 4], [3, 5] + points = []Endpoint{ + {Key: []byte{1}, Tp: InclusiveStart, Weight: 1}, + {Key: []byte{3}, Tp: InclusiveEnd, Weight: 1}, + {Key: []byte{2}, Tp: InclusiveStart, Weight: 1}, + {Key: []byte{4}, Tp: InclusiveEnd, Weight: 1}, + {Key: []byte{3}, Tp: InclusiveStart, Weight: 1}, + {Key: []byte{5}, Tp: InclusiveEnd, Weight: 1}, + } + require.Equal(t, 3, GetMaxOverlapping(points)) +} diff --git a/br/pkg/lightning/backend/external/writer.go b/br/pkg/lightning/backend/external/writer.go index 500e02af81b34..0b5a7897be60e 100644 --- a/br/pkg/lightning/backend/external/writer.go +++ b/br/pkg/lightning/backend/external/writer.go @@ -36,6 +36,8 @@ import ( "go.uber.org/zap" ) +var multiFileStatNum = 500 + // rangePropertiesCollector collects range properties for each range. The zero // value of rangePropertiesCollector is not ready to use, should call reset() // first. @@ -59,11 +61,12 @@ func (rc *rangePropertiesCollector) encode() []byte { // WriterSummary is the summary of a writer. type WriterSummary struct { - WriterID int - Seq int - Min tidbkv.Key - Max tidbkv.Key - TotalSize uint64 + WriterID int + Seq int + Min tidbkv.Key + Max tidbkv.Key + TotalSize uint64 + MultipleFilesStats []MultipleFilesStat } // OnCloseFunc is the callback function when a writer is closed. @@ -155,7 +158,7 @@ func (b *WriterBuilder) Build( if b.dupeDetectEnabled { keyAdapter = common.DupDetectKeyAdapter{} } - return &Writer{ + ret := &Writer{ rc: &rangePropertiesCollector{ props: make([]*rangeProperty, 0, 1024), currProp: &rangeProperty{}, @@ -173,7 +176,47 @@ func (b *WriterBuilder) Build( kvStore: nil, onClose: b.onClose, closed: false, + multiFileStats: make([]MultipleFilesStat, 1), + fileMinKeys: make([]tidbkv.Key, 0, multiFileStatNum), + fileMaxKeys: make([]tidbkv.Key, 0, multiFileStatNum), } + ret.multiFileStats[0].Filenames = make([][2]string, 0, multiFileStatNum) + return ret +} + +// MultipleFilesStat is the statistic information of multiple files (currently +// every 500 files). It is used to estimate the data overlapping, and per-file +// statistic information maybe too big to loaded into memory. +type MultipleFilesStat struct { + MinKey tidbkv.Key + MaxKey tidbkv.Key + Filenames [][2]string // [dataFile, statFile] + MaxOverlappingNum int +} + +func (m *MultipleFilesStat) build(startKeys, endKeys []tidbkv.Key) { + if len(startKeys) == 0 { + return + } + m.MinKey = startKeys[0] + m.MaxKey = endKeys[0] + for i := 1; i < len(startKeys); i++ { + if m.MinKey.Cmp(startKeys[i]) > 0 { + m.MinKey = startKeys[i] + } + if m.MaxKey.Cmp(endKeys[i]) < 0 { + m.MaxKey = endKeys[i] + } + } + + points := make([]Endpoint, 0, len(startKeys)*2) + for _, k := range startKeys { + points = append(points, Endpoint{Key: k, Tp: InclusiveStart, Weight: 1}) + } + for _, k := range endKeys { + points = append(points, Endpoint{Key: k, Tp: InclusiveEnd, Weight: 1}) + } + m.MaxOverlappingNum = GetMaxOverlapping(points) } // Writer is used to write data into external storage. @@ -198,6 +241,11 @@ type Writer struct { // Statistic information per batch. batchSize uint64 + // Statistic information per 500 batches. + multiFileStats []MultipleFilesStat + fileMinKeys []tidbkv.Key + fileMaxKeys []tidbkv.Key + // Statistic information per writer. minKey tidbkv.Key maxKey tidbkv.Key @@ -218,7 +266,7 @@ func (w *Writer) AppendRows(ctx context.Context, _ []string, rows encode.Rows) e w.writeBatch = append(w.writeBatch, common.KvPair{Key: key, Val: val}) if w.batchSize >= w.memSizeLimit { - if err := w.flushKVs(ctx); err != nil { + if err := w.flushKVs(ctx, false); err != nil { return err } } @@ -238,10 +286,12 @@ func (w *Writer) Close(ctx context.Context) (backend.ChunkFlushStatus, error) { } w.closed = true defer w.kvBuffer.Destroy() - err := w.flushKVs(ctx) + err := w.flushKVs(ctx, true) if err != nil { return status(false), err } + // remove the trailing empty MultipleFilesStat + w.multiFileStats = w.multiFileStats[:len(w.multiFileStats)-1] logutil.Logger(ctx).Info("close writer", zap.Int("writerID", w.writerID), @@ -251,11 +301,12 @@ func (w *Writer) Close(ctx context.Context) (backend.ChunkFlushStatus, error) { w.writeBatch = nil w.onClose(&WriterSummary{ - WriterID: w.writerID, - Seq: w.currentSeq, - Min: w.minKey, - Max: w.maxKey, - TotalSize: w.totalSize, + WriterID: w.writerID, + Seq: w.currentSeq, + Min: w.minKey, + Max: w.maxKey, + TotalSize: w.totalSize, + MultipleFilesStats: w.multiFileStats, }) return status(true), nil } @@ -277,13 +328,13 @@ func (s status) Flushed() bool { return bool(s) } -func (w *Writer) flushKVs(ctx context.Context) (err error) { +func (w *Writer) flushKVs(ctx context.Context, fromClose bool) (err error) { if len(w.writeBatch) == 0 { return nil } logger := logutil.Logger(ctx) - dataWriter, statWriter, err := w.createStorageWriter(ctx) + dataFile, statFile, dataWriter, statWriter, err := w.createStorageWriter(ctx) if err != nil { return err } @@ -339,6 +390,23 @@ func (w *Writer) flushKVs(ctx context.Context) (err error) { w.recordMinMax(w.writeBatch[0].Key, w.writeBatch[len(w.writeBatch)-1].Key, kvSize) + // maintain 500-batch statistics + + l := len(w.multiFileStats) + w.multiFileStats[l-1].Filenames = append(w.multiFileStats[l-1].Filenames, + [2]string{dataFile, statFile}, + ) + w.fileMinKeys = append(w.fileMinKeys, tidbkv.Key(w.writeBatch[0].Key).Clone()) + w.fileMaxKeys = append(w.fileMaxKeys, tidbkv.Key(w.writeBatch[len(w.writeBatch)-1].Key).Clone()) + if fromClose || len(w.multiFileStats[l-1].Filenames) == multiFileStatNum { + w.multiFileStats[l-1].build(w.fileMinKeys, w.fileMaxKeys) + w.multiFileStats = append(w.multiFileStats, MultipleFilesStat{ + Filenames: make([][2]string, 0, multiFileStatNum), + }) + w.fileMinKeys = w.fileMinKeys[:0] + w.fileMaxKeys = w.fileMaxKeys[:0] + } + w.writeBatch = w.writeBatch[:0] w.rc.reset() w.kvBuffer.Reset() @@ -347,16 +415,20 @@ func (w *Writer) flushKVs(ctx context.Context) (err error) { return nil } -func (w *Writer) createStorageWriter(ctx context.Context) (data, stats storage.ExternalFileWriter, err error) { +func (w *Writer) createStorageWriter(ctx context.Context) ( + dataFile, statFile string, + data, stats storage.ExternalFileWriter, + err error, +) { dataPath := filepath.Join(w.filenamePrefix, strconv.Itoa(w.currentSeq)) dataWriter, err := w.store.Create(ctx, dataPath, nil) if err != nil { - return nil, nil, err + return "", "", nil, nil, err } statPath := filepath.Join(w.filenamePrefix+statSuffix, strconv.Itoa(w.currentSeq)) statsWriter, err := w.store.Create(ctx, statPath, nil) if err != nil { - return nil, nil, err + return "", "", nil, nil, err } - return dataWriter, statsWriter, nil + return dataPath, statPath, dataWriter, statsWriter, nil } diff --git a/br/pkg/lightning/backend/external/writer_test.go b/br/pkg/lightning/backend/external/writer_test.go index 243d0cb6eb73f..2692978d08792 100644 --- a/br/pkg/lightning/backend/external/writer_test.go +++ b/br/pkg/lightning/backend/external/writer_test.go @@ -212,3 +212,122 @@ func TestWriterDuplicateDetect(t *testing.T) { require.Error(t, err) require.Contains(t, err.Error(), "found duplicate key") } + +func TestMultiFileStat(t *testing.T) { + s := &MultipleFilesStat{} + // [3, 5], [1, 3], [2, 4] + startKeys := []dbkv.Key{{3}, {1}, {2}} + endKeys := []dbkv.Key{{5}, {3}, {4}} + s.build(startKeys, endKeys) + require.EqualValues(t, []byte{1}, s.MinKey) + require.EqualValues(t, []byte{5}, s.MaxKey) + require.EqualValues(t, 3, s.MaxOverlappingNum) +} + +func TestWriterMultiFileStat(t *testing.T) { + oldMultiFileStatNum := multiFileStatNum + t.Cleanup(func() { + multiFileStatNum = oldMultiFileStatNum + }) + multiFileStatNum = 3 + + ctx := context.Background() + memStore := storage.NewMemStorage() + var summary *WriterSummary + + writer := NewWriterBuilder(). + SetPropKeysDistance(2). + SetMemorySizeLimit(20). // 2 KV pair will trigger flush + SetOnCloseFunc(func(s *WriterSummary) { + summary = s + }). + Build(memStore, "/test", 0) + + kvs := make([]common.KvPair, 0, 18) + // [key01, key02], [key03, key04], [key05, key06] + for i := 1; i <= 6; i++ { + kvs = append(kvs, common.KvPair{ + Key: []byte(fmt.Sprintf("key%02d", i)), + Val: []byte("56789"), + }) + } + // [key11, key13], [key12, key15], [key14, key16] + kvs = append(kvs, common.KvPair{ + Key: []byte("key11"), + Val: []byte("56789"), + }) + kvs = append(kvs, common.KvPair{ + Key: []byte("key13"), + Val: []byte("56789"), + }) + kvs = append(kvs, common.KvPair{ + Key: []byte("key12"), + Val: []byte("56789"), + }) + kvs = append(kvs, common.KvPair{ + Key: []byte("key15"), + Val: []byte("56789"), + }) + kvs = append(kvs, common.KvPair{ + Key: []byte("key14"), + Val: []byte("56789"), + }) + kvs = append(kvs, common.KvPair{ + Key: []byte("key16"), + Val: []byte("56789"), + }) + // [key20, key22], [key21, key23], [key22, key24] + for i := 0; i < 3; i++ { + kvs = append(kvs, common.KvPair{ + Key: []byte(fmt.Sprintf("key2%d", i)), + Val: []byte("56789"), + }) + kvs = append(kvs, common.KvPair{ + Key: []byte(fmt.Sprintf("key2%d", i+2)), + Val: []byte("56789"), + }) + } + + rows := kv.MakeRowsFromKvPairs(kvs) + err := writer.AppendRows(ctx, nil, rows) + require.NoError(t, err) + _, err = writer.Close(ctx) + require.NoError(t, err) + + require.Equal(t, 3, len(summary.MultipleFilesStats)) + expected := MultipleFilesStat{ + MinKey: []byte("key01"), + MaxKey: []byte("key06"), + Filenames: [][2]string{ + {"/test/0/0", "/test/0_stat/0"}, + {"/test/0/1", "/test/0_stat/1"}, + {"/test/0/2", "/test/0_stat/2"}, + }, + MaxOverlappingNum: 1, + } + require.Equal(t, expected, summary.MultipleFilesStats[0]) + expected = MultipleFilesStat{ + MinKey: []byte("key11"), + MaxKey: []byte("key16"), + Filenames: [][2]string{ + {"/test/0/3", "/test/0_stat/3"}, + {"/test/0/4", "/test/0_stat/4"}, + {"/test/0/5", "/test/0_stat/5"}, + }, + MaxOverlappingNum: 2, + } + require.Equal(t, expected, summary.MultipleFilesStats[1]) + expected = MultipleFilesStat{ + MinKey: []byte("key20"), + MaxKey: []byte("key24"), + Filenames: [][2]string{ + {"/test/0/6", "/test/0_stat/6"}, + {"/test/0/7", "/test/0_stat/7"}, + {"/test/0/8", "/test/0_stat/8"}, + }, + MaxOverlappingNum: 3, + } + require.Equal(t, expected, summary.MultipleFilesStats[2]) + require.EqualValues(t, "key01", summary.Min) + require.EqualValues(t, "key24", summary.Max) +} From a677716f10ef9374ee7f4e891e2874ae64678eb1 Mon Sep 17 00:00:00 2001 From: wjHuang Date: Fri, 8 Sep 2023 18:10:43 +0800 Subject: [PATCH 2/9] ddl: relax the check in ownerCheckAllVersions (#46752) close pingcap/tidb#46751 --- ddl/syncer/syncer.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/ddl/syncer/syncer.go b/ddl/syncer/syncer.go index 3cdccb0197cc5..c7f13b68d2ead 100644 --- a/ddl/syncer/syncer.go +++ b/ddl/syncer/syncer.go @@ -337,13 +337,15 @@ func (s *schemaVersionSyncer) OwnerCheckAllVersions(ctx context.Context, jobID i if variable.EnableMDL.Load() { for _, kv := range resp.Kvs { key := string(kv.Key) + tidbIDInResp := key[strings.LastIndex(key, "/")+1:] ver, err := strconv.Atoi(string(kv.Value)) if err != nil { logutil.BgLogger().Info("syncer check all versions, convert value to int failed, continue checking.", zap.String("category", "ddl"), zap.String("ddl", string(kv.Key)), zap.String("value", string(kv.Value)), zap.Error(err)) succ = false break } - if int64(ver) < latestVer { + // We need to check if the tidb ID is in the updatedMap, in case that deleting etcd is failed, and tidb server is down. + if int64(ver) < latestVer && updatedMap[tidbIDInResp] != "" { if notMatchVerCnt%intervalCnt == 0 { logutil.BgLogger().Info("syncer check all versions, someone is not synced, continue checking", zap.String("category", "ddl"), zap.String("ddl", string(kv.Key)), zap.Int("currentVer", ver), zap.Int64("latestVer", latestVer)) @@ -352,7 +354,7 @@ func (s *schemaVersionSyncer) OwnerCheckAllVersions(ctx context.Context, jobID i notMatchVerCnt++ break } - delete(updatedMap, key[strings.LastIndex(key, "/")+1:]) + delete(updatedMap, tidbIDInResp) } if len(updatedMap) > 0 { succ = false From 1d55a3c68d208d457d342db8f8cb5b83f8f307b7 Mon Sep 17 00:00:00 2001 From: "TONG, Zhigao" Date: Fri, 8 Sep 2023 18:49:37 +0800 Subject: [PATCH 3/9] executor: fix panic issue when handle `HashAggExec.Close()` (#46662) close pingcap/tidb#41778 --- executor/aggregate/agg_hash_executor.go | 13 ++++---- .../test/issuetest/executor_issue_test.go | 32 +++++++++++++++++++ 2 files changed, 39 insertions(+), 6 deletions(-) diff --git a/executor/aggregate/agg_hash_executor.go b/executor/aggregate/agg_hash_executor.go index faa6aef74600f..83d38c50519f8 100644 --- a/executor/aggregate/agg_hash_executor.go +++ b/executor/aggregate/agg_hash_executor.go @@ -116,10 +116,10 @@ type HashAggExec struct { IsChildReturnEmpty bool // After we support parallel execution for aggregation functions with distinct, // we can remove this attribute. - IsUnparallelExec bool - parallelExecInitialized bool - prepared bool - executed bool + IsUnparallelExec bool + parallelExecValid bool + prepared bool + executed bool memTracker *memory.Tracker // track memory usage. diskTracker *disk.Tracker @@ -168,7 +168,7 @@ func (e *HashAggExec) Close() error { } return firstErr } - if e.parallelExecInitialized { + if e.parallelExecValid { // `Close` may be called after `Open` without calling `Next` in test. if !e.prepared { close(e.inputCh) @@ -192,6 +192,7 @@ func (e *HashAggExec) Close() error { if e.memTracker != nil { e.memTracker.ReplaceBytesUsed(0) } + e.parallelExecValid = false } return e.BaseExecutor.Close() } @@ -327,7 +328,7 @@ func (e *HashAggExec) initForParallelExec(_ sessionctx.Context) { e.finalWorkers[i].finalResultHolderCh <- exec.NewFirstChunk(e) } - e.parallelExecInitialized = true + e.parallelExecValid = true } // Next implements the Executor Next interface. diff --git a/executor/test/issuetest/executor_issue_test.go b/executor/test/issuetest/executor_issue_test.go index e06cba19c8adb..c261289c31105 100644 --- a/executor/test/issuetest/executor_issue_test.go +++ b/executor/test/issuetest/executor_issue_test.go @@ -1426,3 +1426,35 @@ func TestIssue42662(t *testing.T) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/issue42662_1")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/util/servermemorylimit/issue42662_2")) } + +func TestIssue41778(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(` + CREATE TABLE ta ( + a1 json DEFAULT NULL, + a2 decimal(31, 1) DEFAULT '0' + ); + CREATE TABLE tb ( + b1 smallint(6) DEFAULT '-11385', + b2 decimal(63, 14) DEFAULT '-6197127648752447138876497216172307937317445669286.98661563645110' + ); + CREATE TABLE tc ( + c1 text DEFAULT NULL, + c2 float NOT NULL DEFAULT '1.8132474', + PRIMARY KEY (c2) + /*T![clustered_index] CLUSTERED */ + ); + `) + tk.MustExec(` + insert into ta + values (NULL, 1228.0); + insert into ta + values ('"json string1"', 623.8); + insert into ta + values (NULL, 1337.0); + `) + err := tk.QueryToErr("select count(*)from ta where not ( ta.a1 in ( select b2 from tb where not ( ta.a1 in ( select c1 from tc where ta.a2 in ( select b2 from tb where IsNull(ta.a1) ) ) ) ) )") + require.Equal(t, "[planner:1815]expression isnull(cast(test.ta.a1, var_string(4294967295))) cannot be pushed down", err.Error()) +} From 273816051db70cee5dedc8a944607ed34ca6c25b Mon Sep 17 00:00:00 2001 From: wuhuizuo Date: Fri, 8 Sep 2023 19:25:35 +0800 Subject: [PATCH 4/9] CI: disable codecov's reports in commit status and github cheks (#46232) --- .codecov.yml | 28 ++++++++++++++++++---------- 1 file changed, 18 insertions(+), 10 deletions(-) diff --git a/.codecov.yml b/.codecov.yml index 45b7bb4093f39..69fdbb3af1369 100644 --- a/.codecov.yml +++ b/.codecov.yml @@ -8,15 +8,23 @@ coverage: round: down range: "65...90" + # status: + # project: + # default: + # threshold: 0.2 #Allow the coverage to drop by threshold%, and posting a success status. + # patch: + # default: + # target: 0% # trial operation + + # Ref: https://docs.codecov.com/docs/commit-status#disabling-a-status status: - project: - default: - threshold: 0.2 #Allow the coverage to drop by threshold%, and posting a success status. - patch: - default: - target: 0% # trial operation + project: off # disable it + patch: off # disable it changes: no +# Ref: https://docs.codecov.com/docs/github-checks#disabling-github-checks-completely-via-yaml +github_checks: false + parsers: gcov: branch_detection: @@ -43,7 +51,7 @@ component_management: - component_id: component_parser name: parser paths: - - parser/** + - parser/** - component_id: component_br name: br paths: @@ -52,7 +60,7 @@ component_management: flag_management: default_rules: # the rules that will be followed for any flag added, generally - carryforward: false + carryforward: true statuses: - type: project target: auto @@ -61,11 +69,11 @@ flag_management: ignore: - "LICENSES" - - "*_test.go" + - "*_test.go" - ".git" - "*.yml" - "*.md" - - "cmd/.*" + - "cmd/.*" - "docs/.*" - "vendor/.*" - "ddl/failtest/.*" From 58a2dad97ff4380657792bba2cdbc5660e6bdd4e Mon Sep 17 00:00:00 2001 From: lance6716 Date: Fri, 8 Sep 2023 19:25:43 +0800 Subject: [PATCH 5/9] test: close duplicateDB when meet error halfway (#46685) close pingcap/tidb#46682 --- br/pkg/lightning/backend/local/local.go | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 56e5e68eac30c..2d54688f96928 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -506,7 +506,13 @@ func NewBackend( tls *common.TLS, config BackendConfig, regionSizeGetter TableRegionSizeGetter, -) (*Backend, error) { +) (b *Backend, err error) { + var duplicateDB *pebble.DB + defer func() { + if err != nil && duplicateDB != nil { + _ = duplicateDB.Close() + } + }() config.adjust() pdCtl, err := pdutil.NewPdController(ctx, config.PDAddr, tls.TLSConfig(), tls.ToPDSecurityOption()) if err != nil { @@ -532,7 +538,6 @@ func NewBackend( } } - var duplicateDB *pebble.DB if config.DupeDetectEnabled { duplicateDB, err = openDuplicateDB(config.LocalStoreDir) if err != nil { From 968c7129bcf4e501137fad9c9b03355cfad9f85f Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Fri, 8 Sep 2023 20:00:06 +0800 Subject: [PATCH 6/9] sessionctx: add testcase for set_var hint (#46784) ref pingcap/tidb#45892 --- sessionctx/variable/session.go | 7 ++++++- sessionctx/variable/setvar_test.go | 20 ++++++++++++++++++++ 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index b903090472b76..9b8eeae24ca0e 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -2536,7 +2536,12 @@ func (s *SessionVars) SetSystemVarWithOldValAsRet(name string, val string) (stri if err != nil { return "", err } - oldV := sv.Value + // The map s.systems[sv.Name] is lazy initialized. If we directly read it, we might read empty result. + // Since this code path is not a hot path, we directly call GetSessionOrGlobalSystemVar to get the value safely. + oldV, err := s.GetSessionOrGlobalSystemVar(context.Background(), sv.Name) + if err != nil { + return "", err + } return oldV, sv.SetSessionFromHook(s, val) } diff --git a/sessionctx/variable/setvar_test.go b/sessionctx/variable/setvar_test.go index eb8eaa11e15b2..4e5db8b274ff9 100644 --- a/sessionctx/variable/setvar_test.go +++ b/sessionctx/variable/setvar_test.go @@ -110,6 +110,26 @@ func TestSetVarNonStringOrEnum(t *testing.T) { tk.MustQuery(fmt.Sprintf("select @@%v", c.varName)).Check(r) } } + + tk.MustQuery("select @@max_execution_time").Check(testkit.Rows("0")) + tk.MustExec("set @@max_execution_time=1000") + tk.MustQuery("select @@max_execution_time").Check(testkit.Rows("1000")) + tk.MustQuery("select /*+ set_var(max_execution_time=100) */ @@max_execution_time").Check(testkit.Rows("100")) + // The value is the changed value 1000, not the default value 0. + tk.MustQuery("select @@max_execution_time").Check(testkit.Rows("1000")) + + tk.MustExec("set @@global.max_execution_time=1000") + + tk2 := testkit.NewTestKit(t, store) + tk2.MustQuery("select @@max_execution_time").Check(testkit.Rows("1000")) + tk2.MustQuery("select /*+ set_var(max_execution_time=100) */ @@max_execution_time").Check(testkit.Rows("100")) + // The value is the global value 1000, not the default value 0. + tk2.MustQuery("select @@max_execution_time").Check(testkit.Rows("1000")) + tk2.MustExec("set @@max_execution_time=2000") + tk2.MustQuery("select @@max_execution_time").Check(testkit.Rows("2000")) + tk2.MustQuery("select /*+ set_var(max_execution_time=100) */ @@max_execution_time").Check(testkit.Rows("100")) + // The value is the changed value 2000, not the default value 0 or the global value 1000. + tk2.MustQuery("select @@max_execution_time").Check(testkit.Rows("2000")) } func TestSetVarStringOrEnum(t *testing.T) { From a41f5ea2e2894a7d7100d76d9f8588171f2bfb29 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Fri, 8 Sep 2023 22:48:07 +0800 Subject: [PATCH 7/9] planner: split index test (#46780) close pingcap/tidb#44940 --- cmd/mirror/mirror.go | 4 +- executor/distsql_test.go | 17 - planner/core/casetest/BUILD.bazel | 1 - planner/core/casetest/index/BUILD.bazel | 23 + planner/core/casetest/index/index_test.go | 419 ++++++ planner/core/casetest/index/main_test.go | 54 + .../index/testdata/integration_suite_in.json | 153 +++ .../index/testdata/integration_suite_out.json | 1079 +++++++++++++++ planner/core/casetest/integration_test.go | 364 ----- planner/core/casetest/pushdown/BUILD.bazel | 1 - .../core/casetest/pushdown/push_down_test.go | 12 + .../testdata/integration_suite_in.json | 164 --- .../testdata/integration_suite_out.json | 1180 +---------------- 13 files changed, 1771 insertions(+), 1700 deletions(-) create mode 100644 planner/core/casetest/index/BUILD.bazel create mode 100644 planner/core/casetest/index/index_test.go create mode 100644 planner/core/casetest/index/main_test.go create mode 100644 planner/core/casetest/index/testdata/integration_suite_in.json create mode 100644 planner/core/casetest/index/testdata/integration_suite_out.json diff --git a/cmd/mirror/mirror.go b/cmd/mirror/mirror.go index 10e98097b6e53..38b335e7be860 100644 --- a/cmd/mirror/mirror.go +++ b/cmd/mirror/mirror.go @@ -197,7 +197,7 @@ func downloadZips( cmd := exec.Command(gobin, downloadArgs...) cmd.Dir = tmpdir env := os.Environ() - env = append(env, fmt.Sprintf("GOPROXY=%s", "https://proxy.golang.org,direct")) + env = append(env, fmt.Sprintf("GOPROXY=%s", "https://mirrors.aliyun.com/goproxy/,https://proxy.golang.org,direct")) env = append(env, fmt.Sprintf("GOSUMDB=%s", "sum.golang.org")) cmd.Env = env jsonBytes, err := cmd.Output() @@ -228,7 +228,7 @@ func listAllModules(tmpdir string) (map[string]listedModule, error) { cmd := exec.Command(gobin, "list", "-mod=readonly", "-m", "-json", "all") cmd.Dir = tmpdir env := os.Environ() - env = append(env, fmt.Sprintf("GOPROXY=%s", "https://proxy.golang.org,direct")) + env = append(env, fmt.Sprintf("GOPROXY=%s", "https://mirrors.aliyun.com/goproxy/,https://proxy.golang.org,direct")) env = append(env, fmt.Sprintf("GOSUMDB=%s", "sum.golang.org")) cmd.Env = env jsonBytes, err := cmd.Output() diff --git a/executor/distsql_test.go b/executor/distsql_test.go index e88464e0c739d..c00289ccb5c8d 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -273,23 +273,6 @@ func TestInconsistentIndex(t *testing.T) { } } -func TestPushLimitDownIndexLookUpReader(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists tbl") - tk.MustExec("create table tbl(a int, b int, c int, key idx_b_c(b,c))") - tk.MustExec("insert into tbl values(1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5)") - tk.MustQuery("select * from tbl use index(idx_b_c) where b > 1 limit 2,1").Check(testkit.Rows("4 4 4")) - tk.MustQuery("select * from tbl use index(idx_b_c) where b > 4 limit 2,1").Check(testkit.Rows()) - tk.MustQuery("select * from tbl use index(idx_b_c) where b > 3 limit 2,1").Check(testkit.Rows()) - tk.MustQuery("select * from tbl use index(idx_b_c) where b > 2 limit 2,1").Check(testkit.Rows("5 5 5")) - tk.MustQuery("select * from tbl use index(idx_b_c) where b > 1 limit 1").Check(testkit.Rows("2 2 2")) - tk.MustQuery("select * from tbl use index(idx_b_c) where b > 1 order by b desc limit 2,1").Check(testkit.Rows("3 3 3")) - tk.MustQuery("select * from tbl use index(idx_b_c) where b > 1 and c > 1 limit 2,1").Check(testkit.Rows("4 4 4")) -} - func TestPartitionTableIndexLookUpReader(t *testing.T) { failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") diff --git a/planner/core/casetest/BUILD.bazel b/planner/core/casetest/BUILD.bazel index 783daa7fb0a28..b1ae48368c0b0 100644 --- a/planner/core/casetest/BUILD.bazel +++ b/planner/core/casetest/BUILD.bazel @@ -29,7 +29,6 @@ go_test( "//testkit/testdata", "//testkit/testmain", "//testkit/testsetup", - "//util", "//util/hint", "//util/plancodec", "@com_github_pingcap_failpoint//:failpoint", diff --git a/planner/core/casetest/index/BUILD.bazel b/planner/core/casetest/index/BUILD.bazel new file mode 100644 index 0000000000000..b6b88e1899b8b --- /dev/null +++ b/planner/core/casetest/index/BUILD.bazel @@ -0,0 +1,23 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "index_test", + timeout = "short", + srcs = [ + "index_test.go", + "main_test.go", + ], + data = glob(["testdata/**"]), + flaky = True, + shard_count = 12, + deps = [ + "//sessionctx/variable", + "//testkit", + "//testkit/testdata", + "//testkit/testmain", + "//testkit/testsetup", + "//util", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/planner/core/casetest/index/index_test.go b/planner/core/casetest/index/index_test.go new file mode 100644 index 0000000000000..29233f93496d9 --- /dev/null +++ b/planner/core/casetest/index/index_test.go @@ -0,0 +1,419 @@ +// 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 index + +import ( + "fmt" + "strings" + "testing" + + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/util" + "github.com/stretchr/testify/require" +) + +func TestIndexJoinUniqueCompositeIndex(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.MustExec("create table t1(a int not null, c int not null)") + tk.MustExec("create table t2(a int not null, b int not null, c int not null, primary key(a,b))") + tk.MustExec("insert into t1 values(1,1)") + tk.MustExec("insert into t2 values(1,1,1),(1,2,1)") + tk.MustExec("analyze table t1,t2") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestIndexMerge(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, c int, unique index(a), unique index(b), primary key(c))") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +// for issue #14822 and #38258 +func TestIndexJoinTableRange(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int, b int, primary key (a), key idx_t1_b (b))") + tk.MustExec("create table t2(a int, b int, primary key (a), key idx_t1_b (b))") + tk.MustExec("create table t3(a int, b int, c int)") + tk.MustExec("create table t4(a int, b int, c int, primary key (a, b) clustered)") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestIndexJoinInnerIndexNDV(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int not null, b int not null, c int not null)") + tk.MustExec("create table t2(a int not null, b int not null, c int not null, index idx1(a,b), index idx2(c))") + tk.MustExec("insert into t1 values(1,1,1),(1,1,1),(1,1,1)") + tk.MustExec("insert into t2 values(1,1,1),(1,1,2),(1,1,3)") + tk.MustExec("analyze table t1, t2") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestIndexMergeSerial(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int, unique key(a), unique key(b))") + tk.MustExec("insert into t value (1, 5), (2, 4), (3, 3), (4, 2), (5, 1)") + tk.MustExec("insert into t value (6, 0), (7, -1), (8, -2), (9, -3), (10, -4)") + tk.MustExec("analyze table t") + + var input []string + var output []struct { + SQL string + Plan []string + Warnings []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) + } +} + +func TestIndexJoinOnClusteredIndex(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t (a int, b varchar(20), c decimal(40,10), d int, primary key(a,b), key(c))") + tk.MustExec(`insert into t values (1,"111",1.1,11), (2,"222",2.2,12), (3,"333",3.3,13)`) + tk.MustExec("analyze table t") + + var input []string + var output []struct { + SQL string + Plan []string + Res []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery("explain format = 'brief'" + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) + } +} + +func TestIndexMergeWithCorrelatedColumns(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("create table t1(c1 int, c2 int, c3 int, primary key(c1), key(c2));") + tk.MustExec("insert into t1 values(1, 1, 1);") + tk.MustExec("insert into t1 values(2, 2, 2);") + tk.MustExec("create table t2(c1 int, c2 int, c3 int);") + tk.MustExec("insert into t2 values(1, 1, 1);") + tk.MustExec("insert into t2 values(2, 2, 2);") + + tk.MustExec("drop table if exists tt1, tt2;") + tk.MustExec("create table tt1 (c_int int, c_str varchar(40), c_datetime datetime, c_decimal decimal(12, 6), primary key(c_int), key(c_int), key(c_str), unique key(c_decimal), key(c_datetime));") + tk.MustExec("create table tt2 like tt1 ;") + tk.MustExec(`insert into tt1 (c_int, c_str, c_datetime, c_decimal) values (6, 'sharp payne', '2020-06-07 10:40:39', 6.117000) , + (7, 'objective kare', '2020-02-05 18:47:26', 1.053000) , + (8, 'thirsty pasteur', '2020-01-02 13:06:56', 2.506000) , + (9, 'blissful wilbur', '2020-06-04 11:34:04', 9.144000) , + (10, 'reverent mclean', '2020-02-12 07:36:26', 7.751000) ;`) + tk.MustExec(`insert into tt2 (c_int, c_str, c_datetime, c_decimal) values (6, 'beautiful joliot', '2020-01-16 01:44:37', 5.627000) , + (7, 'hopeful blackburn', '2020-05-23 21:44:20', 7.890000) , + (8, 'ecstatic davinci', '2020-02-01 12:27:17', 5.648000) , + (9, 'hopeful lewin', '2020-05-05 05:58:25', 7.288000) , + (10, 'sharp jennings', '2020-01-28 04:35:03', 9.758000) ;`) + + var input []string + var output []struct { + SQL string + Plan []string + Res []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format=brief " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery("explain format=brief " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) + } +} + +func TestIndexJoinRangeFallback(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int, b int, c varchar(10), d varchar(10), index idx_a_b_c_d(a, b, c(2), d(2)))") + tk.MustExec("create table t2(e int, f int, g varchar(10), h varchar(10))") + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + setStmt := strings.HasPrefix(tt, "set") + testdata.OnRecord(func() { + output[i].SQL = tt + if !setStmt { + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + } + }) + if setStmt { + tk.MustExec(tt) + } else { + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } + } +} + +func TestNullConditionForPrefixIndex(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`CREATE TABLE t1 ( + id char(1) DEFAULT NULL, + c1 varchar(255) DEFAULT NULL, + c2 text DEFAULT NULL, + KEY idx1 (c1), + KEY idx2 (c1,c2(5)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin`) + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("create table t2(a int, b varchar(10), index idx(b(5)))") + tk.MustExec("create table t3(a int, b varchar(10), c int, primary key (a, b(5)) clustered)") + tk.MustExec("set tidb_opt_prefix_index_single_scan = 1") + tk.MustExec("insert into t1 values ('a', '0xfff', '111111'), ('b', '0xfff', '22 '), ('c', '0xfff', ''), ('d', '0xfff', null)") + tk.MustExec("insert into t2 values (1, 'aaaaaa'), (2, 'bb '), (3, ''), (4, null)") + tk.MustExec("insert into t3 values (1, 'aaaaaa', 2), (1, 'bb ', 3), (1, '', 4)") + + var input []string + var output []struct { + SQL string + Plan []string + Result []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + tt).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + }) + tk.MustQuery("explain format='brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Result...)) + } + + // test plan cache + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk.MustExec("set @@tidb_enable_collect_execution_info=0") + tk.MustExec("prepare stmt from 'select count(1) from t1 where c1 = ? and c2 is not null'") + tk.MustExec("set @a = '0xfff'") + tk.MustQuery("execute stmt using @a").Check(testkit.Rows("3")) + tk.MustQuery("execute stmt using @a").Check(testkit.Rows("3")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustQuery("execute stmt using @a").Check(testkit.Rows("3")) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( + "StreamAgg_17 1.00 root funcs:count(Column#7)->Column#5", + "└─IndexReader_18 1.00 root index:StreamAgg_9", + " └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_16 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo")) +} + +func TestHeuristicIndexSelection(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int, b int, c int, d int, e int, f int, g int, primary key (a), unique key c_d_e (c, d, e), unique key f (f), unique key f_g (f, g), key g (g))") + tk.MustExec("create table t2(a int, b int, c int, d int, unique index idx_a (a), unique index idx_b_c (b, c), unique index idx_b_c_a_d (b, c, a, d))") + tk.MustExec("create table t3(a bigint, b varchar(255), c bigint, primary key(a, b) clustered)") + tk.MustExec("create table t4(a bigint, b varchar(255), c bigint, primary key(a, b) nonclustered)") + + // Default RPC encoding may cause statistics explain result differ and then the test unstable. + tk.MustExec("set @@tidb_enable_chunk_rpc = on") + + var input []string + var output []struct { + SQL string + Plan []string + Warnings []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'verbose' " + tt).Rows()) + output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + }) + tk.MustQuery("explain format = 'verbose' " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) + } +} + +func TestLimitIndexLookUpKeepOrder(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int, b int, c int, d int, index idx(a,b,c));") + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestAccessPathOnClusterIndex(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1 (a int, b varchar(20), c decimal(40,10), d int, primary key(a,b), key(c))") + tk.MustExec(`insert into t1 values (1,"111",1.1,11), (2,"222",2.2,12), (3,"333",3.3,13)`) + tk.MustExec("analyze table t1") + + var input []string + var output []struct { + SQL string + Plan []string + Res []string + } + integrationSuiteData := GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + }) + tk.MustQuery("explain format='brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) + } +} diff --git a/planner/core/casetest/index/main_test.go b/planner/core/casetest/index/main_test.go new file mode 100644 index 0000000000000..52c9296a142b8 --- /dev/null +++ b/planner/core/casetest/index/main_test.go @@ -0,0 +1,54 @@ +// 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 index + +import ( + "flag" + "testing" + + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/testkit/testmain" + "github.com/pingcap/tidb/testkit/testsetup" + "go.uber.org/goleak" +) + +var testDataMap = make(testdata.BookKeeper) + +func TestMain(m *testing.M) { + testsetup.SetupForCommonTest() + + flag.Parse() + testDataMap.LoadTestSuiteData("testdata", "integration_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("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), + goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/txnkv/transaction.keepAlive"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + + callback := func(i int) int { + testDataMap.GenerateOutputIfNeeded() + return i + } + + goleak.VerifyTestMain(testmain.WrapTestingM(m, callback), opts...) +} + +func GetIntegrationSuiteData() testdata.TestData { + return testDataMap["integration_suite"] +} diff --git a/planner/core/casetest/index/testdata/integration_suite_in.json b/planner/core/casetest/index/testdata/integration_suite_in.json new file mode 100644 index 0000000000000..7ad4702ef900c --- /dev/null +++ b/planner/core/casetest/index/testdata/integration_suite_in.json @@ -0,0 +1,153 @@ +[ + { + "name": "TestIndexJoinInnerIndexNDV", + "cases": [ + // t2 should use idx2 instead of idx1, since idx2 has larger NDV. + "explain format = 'brief' select /*+ inl_join(t2) */ * from t1, t2 where t1.a = t2.a and t1.b = t2.b and t1.c = t2.c" + ] + }, + { + "name": "TestIndexJoinUniqueCompositeIndex", + "cases": [ + // Row count of IndexScan should be 2. + "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c = t2.c", + // Row count of IndexScan should be 2. + "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c <= t2.b", + // Row count of IndexScan should be 1. + "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t2.b = 1" + ] + }, + { + "name": "TestIndexMerge", + "cases": [ + "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 1 or b = 2", + "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, A, B) */ * from t where a = 1 or b = 2", + "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, primary) */ * from t where 1 or t.c", + "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b, c) */ * from t where 1 or t.a = 1 or t.b = 2" + ] + }, + { + "name": "TestIndexJoinTableRange", + "cases": [ + "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.b", + "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.a and t1.b = t2.b", + "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.a = t4.a where t4.b = 1", + "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.b = t4.b where t4.a = 1" + ] + }, + { + "name": "TestAccessPathOnClusterIndex", + "cases": [ + "select * from t1", + "select * from t1 where t1.a >= 1 and t1.a < 4", + "select * from t1 where t1.a = 1 and t1.b < \"333\"", + "select t1.a, t1.b, t1.c from t1 where t1.c = 3.3", + "select t1.b, t1.c from t1 where t1.c = 2.2", + "select /*+ use_index(t1, c) */ * from t1", + "select * from t1 use index(c) where t1.c in (2.2, 3.3)", + "select * from t1 where t1.a = 1 order by b", + "select * from t1 order by a, b limit 1", + "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a >= 1 or t1.c = 2.2", + "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a = 1 and t1.b = '111' or t1.c = 3.3" + ] + }, + { + "name": "TestIndexJoinOnClusteredIndex", + "cases": [ + "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", + "select /*+ inl_merge_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", + "select /*+ inl_hash_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", + "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a and t1.b = t2.b", + "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.c = t2.c", + "select /*+ inl_merge_join(t1,t2) */ t2.a, t2.c, t2.d from t t1 left join t t2 on t1.a = t2.c;" + ] + }, + { + "name": "TestHeuristicIndexSelection", + "cases": [ + "select * from t1 where a = 3 or a = 5", + "select f, g from t1 where f = 2 and g in (3, 4, 5)", + "select * from t1 where c = 1 and (d = 2 or d = 3) and e in (4, 5)", + "select f, g from t1 where f = 2 and g > 3", + "select a, b, c from t2 where a = 1 and b = 2 and c in (1, 2, 3, 4, 5)", + "select * from t3 where (a = 1 or a = 3) and b = 'xx'", + "select * from t4 where (a = 1 or a = 3) and b = 'xx'", + "select a, b from t3 where (a = 1 or a = 3) and b = 'xx'", + "select a, b from t4 where (a = 1 or a = 3) and b = 'xx'", + "update t1 set b = 2 where a = 4 or a = 6", + "delete from t1 where f = 2 and g in (3, 4)", + "insert into t3 select a, b, c from t1 where f = 2", + "replace into t3 select a, b, c from t1 where a = 3" + ] + }, + { + "name": "TestIndexMergeWithCorrelatedColumns", + "cases": [ + "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", + "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and reverse(c3)) order by c1;", + "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 >= 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", + // Test correlated column in IndexPath.TableFilters. + "select c_int from tt1 where c_decimal < all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt1.c_int = tt2.c_int and tt1.c_datetime > tt2.c_datetime and tt2.c_decimal = 9.060 or tt2.c_str <= 'interesting shtern' and tt1.c_int = tt2.c_int) order by 1;", + // Test correlated column in TablePath.TableFilters. + "select c_int from tt1 where c_decimal > all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt2.c_int = 7 and tt2.c_int < tt1.c_decimal or tt2.c_str >= 'zzzzzzzzzzzzzzzzzzz' and tt1.c_int = tt2.c_int) order by 1;" + ] + }, + { + "name": "TestIndexMergeSerial", + "cases": [ + "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", + "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", + "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", + "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)" + ] + }, + { + "name": "TestLimitIndexLookUpKeepOrder", + "cases": [ + "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", + "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10" + ] + }, + { + "name": "TestIndexJoinRangeFallback", + "cases": [ + "set @@tidb_opt_range_max_size = 0", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "set @@tidb_opt_range_max_size = 2900", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "set @@tidb_opt_range_max_size = 2300", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "set @@tidb_opt_range_max_size = 700", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "set @@tidb_opt_range_max_size = 0", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", + "set @@tidb_opt_range_max_size = 300", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", + "set @@tidb_opt_range_max_size = 0", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10", + "set @@tidb_opt_range_max_size = 300", + "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10" + ] + }, + { + "name": "TestNullConditionForPrefixIndex", + "cases": [ + "select count(1) from t1 where c1 = '0xfff' and c2 is not null", + "select count(1) from t1 where c1 = '0xfff' and c2 is null", + "select count(1) from t1 where c1 >= '0xfff' and c2 is not null", + "select count(1) from t1 where c1 >= '0xfff' and c2 is null", + "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is not null", + "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is null", + "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is not null", + "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is null", + "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is not null", + "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is null", + "select count(1) from t2 use index(idx) where b is not null", + "select count(1) from t2 use index(idx) where b is null", + "select b from t2 use index(idx) where b is not null", + "select b from t2 use index(idx) where b is null", + "select b from t3 where a = 1 and b is not null", + "select b from t3 where a = 1 and b is null" + ] + } +] diff --git a/planner/core/casetest/index/testdata/integration_suite_out.json b/planner/core/casetest/index/testdata/integration_suite_out.json new file mode 100644 index 0000000000000..6d8455d198022 --- /dev/null +++ b/planner/core/casetest/index/testdata/integration_suite_out.json @@ -0,0 +1,1079 @@ +[ + { + "Name": "TestIndexJoinInnerIndexNDV", + "Cases": [ + { + "SQL": "explain format = 'brief' select /*+ inl_join(t2) */ * from t1, t2 where t1.a = t2.a and t1.b = t2.b and t1.c = t2.c", + "Plan": [ + "IndexJoin 3.00 root inner join, inner:IndexLookUp, outer key:test.t1.c, inner key:test.t2.c, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.b, test.t2.b), eq(test.t1.c, test.t2.c)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", + "└─IndexLookUp(Probe) 3.00 root ", + " ├─IndexRangeScan(Build) 3.00 cop[tikv] table:t2, index:idx2(c) range: decided by [eq(test.t2.c, test.t1.c)], keep order:false", + " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t2 keep order:false" + ] + } + ] + }, + { + "Name": "TestIndexJoinUniqueCompositeIndex", + "Cases": [ + { + "SQL": "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c = t2.c", + "Plan": [ + "IndexJoin 2.00 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.c, test.t2.c)", + "├─TableReader(Build) 1.00 root data:TableFullScan", + "│ └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false", + "└─IndexLookUp(Probe) 2.00 root ", + " ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false", + " └─TableRowIDScan(Probe) 2.00 cop[tikv] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c <= t2.b", + "Plan": [ + "IndexJoin 2.00 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a), other cond:le(test.t1.c, test.t2.b)", + "├─TableReader(Build) 1.00 root data:TableFullScan", + "│ └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false", + "└─IndexLookUp(Probe) 2.00 root ", + " ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(test.t2.a, test.t1.a) le(test.t1.c, test.t2.b)], keep order:false", + " └─TableRowIDScan(Probe) 2.00 cop[tikv] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t2.b = 1", + "Plan": [ + "IndexJoin 1.00 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "├─TableReader(Build) 1.00 root data:TableFullScan", + "│ └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false", + "└─IndexLookUp(Probe) 1.00 root ", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(test.t2.a, test.t1.a) eq(test.t2.b, 1)], keep order:false", + " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2 keep order:false" + ] + } + ] + }, + { + "Name": "TestIndexMerge", + "Cases": [ + { + "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 1 or b = 2", + "Plan": [ + "IndexMerge 2.00 root type: union", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[2,2], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, A, B) */ * from t where a = 1 or b = 2", + "Plan": [ + "IndexMerge 2.00 root type: union", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[2,2], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, primary) */ * from t where 1 or t.c", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b, c) */ * from t where 1 or t.a = 1 or t.b = 2", + "Plan": [ + "TableReader 10000.00 root data:Selection", + "└─Selection 10000.00 cop[tikv] or(1, or(eq(test.t.a, 1), eq(test.t.b, 2)))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIndexJoinTableRange", + "Cases": [ + { + "SQL": "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.b", + "Plan": [ + "IndexJoin 12487.50 root inner join, inner:TableReader, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.b, test.t2.b)", + "├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:idx_t1_b(b) keep order:false, stats:pseudo", + "└─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t2.b))", + " └─TableRangeScan 9990.00 cop[tikv] table:t2 range: decided by [test.t1.a], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.a and t1.b = t2.b", + "Plan": [ + "IndexJoin 12487.50 root inner join, inner:TableReader, outer key:test.t1.a, test.t1.b, inner key:test.t2.a, test.t2.a, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.b, test.t2.a), eq(test.t1.b, test.t2.b)", + "├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:idx_t1_b(b) keep order:false, stats:pseudo", + "└─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t2.b))", + " └─TableRangeScan 9990.00 cop[tikv] table:t2 range: decided by [test.t1.a test.t1.b], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.a = t4.a where t4.b = 1", + "Plan": [ + "Projection 12.50 root test.t3.a, test.t3.b, test.t3.c, test.t4.a, test.t4.b, test.t4.c", + "└─IndexJoin 12.50 root inner join, inner:TableReader, outer key:test.t3.a, inner key:test.t4.a, equal cond:eq(test.t3.a, test.t4.a)", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9.99 root data:Selection", + " └─Selection 9.99 cop[tikv] eq(test.t4.b, 1)", + " └─TableRangeScan 9990.00 cop[tikv] table:t4 range: decided by [eq(test.t4.a, test.t3.a) eq(test.t4.b, 1)], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.b = t4.b where t4.a = 1", + "Plan": [ + "Projection 12.50 root test.t3.a, test.t3.b, test.t3.c, test.t4.a, test.t4.b, test.t4.c", + "└─IndexJoin 12.50 root inner join, inner:TableReader, outer key:test.t3.b, inner key:test.t4.b, equal cond:eq(test.t3.b, test.t4.b)", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9.99 root data:Selection", + " └─Selection 9.99 cop[tikv] eq(test.t4.a, 1)", + " └─TableRangeScan 9990.00 cop[tikv] table:t4 range: decided by [eq(test.t4.b, test.t3.b) eq(test.t4.a, 1)], keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestAccessPathOnClusterIndex", + "Cases": [ + { + "SQL": "select * from t1", + "Plan": [ + "TableReader 3.00 root data:TableFullScan", + "└─TableFullScan 3.00 cop[tikv] table:t1 keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11", + "2 222 2.2000000000 12", + "3 333 3.3000000000 13" + ] + }, + { + "SQL": "select * from t1 where t1.a >= 1 and t1.a < 4", + "Plan": [ + "TableReader 3.00 root data:TableRangeScan", + "└─TableRangeScan 3.00 cop[tikv] table:t1 range:[1,4), keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11", + "2 222 2.2000000000 12", + "3 333 3.3000000000 13" + ] + }, + { + "SQL": "select * from t1 where t1.a = 1 and t1.b < \"333\"", + "Plan": [ + "TableReader 0.82 root data:TableRangeScan", + "└─TableRangeScan 0.82 cop[tikv] table:t1 range:[1 -inf,1 \"333\"), keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11" + ] + }, + { + "SQL": "select t1.a, t1.b, t1.c from t1 where t1.c = 3.3", + "Plan": [ + "IndexReader 1.00 root index:IndexRangeScan", + "└─IndexRangeScan 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false" + ], + "Res": [ + "3 333 3.3000000000" + ] + }, + { + "SQL": "select t1.b, t1.c from t1 where t1.c = 2.2", + "Plan": [ + "IndexReader 1.00 root index:IndexRangeScan", + "└─IndexRangeScan 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false" + ], + "Res": [ + "222 2.2000000000" + ] + }, + { + "SQL": "select /*+ use_index(t1, c) */ * from t1", + "Plan": [ + "IndexLookUp 3.00 root ", + "├─IndexFullScan(Build) 3.00 cop[tikv] table:t1, index:c(c) keep order:false", + "└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t1 keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11", + "2 222 2.2000000000 12", + "3 333 3.3000000000 13" + ] + }, + { + "SQL": "select * from t1 use index(c) where t1.c in (2.2, 3.3)", + "Plan": [ + "IndexLookUp 2.00 root ", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], [3.3000000000,3.3000000000], keep order:false", + "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t1 keep order:false" + ], + "Res": [ + "2 222 2.2000000000 12", + "3 333 3.3000000000 13" + ] + }, + { + "SQL": "select * from t1 where t1.a = 1 order by b", + "Plan": [ + "TableReader 1.00 root data:TableRangeScan", + "└─TableRangeScan 1.00 cop[tikv] table:t1 range:[1,1], keep order:true" + ], + "Res": [ + "1 111 1.1000000000 11" + ] + }, + { + "SQL": "select * from t1 order by a, b limit 1", + "Plan": [ + "Limit 1.00 root offset:0, count:1", + "└─TableReader 1.00 root data:Limit", + " └─Limit 1.00 cop[tikv] offset:0, count:1", + " └─TableFullScan 1.00 cop[tikv] table:t1 keep order:true" + ], + "Res": [ + "1 111 1.1000000000 11" + ] + }, + { + "SQL": "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a >= 1 or t1.c = 2.2", + "Plan": [ + "IndexMerge 3.00 root type: union", + "├─TableRangeScan(Build) 3.00 cop[tikv] table:t1 range:[1,+inf], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false", + "└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t1 keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11", + "2 222 2.2000000000 12", + "3 333 3.3000000000 13" + ] + }, + { + "SQL": "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a = 1 and t1.b = '111' or t1.c = 3.3", + "Plan": [ + "IndexMerge 1.67 root type: union", + "├─TableRangeScan(Build) 1.00 cop[tikv] table:t1 range:[1 \"111\",1 \"111\"], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false", + "└─TableRowIDScan(Probe) 1.67 cop[tikv] table:t1 keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11", + "3 333 3.3000000000 13" + ] + } + ] + }, + { + "Name": "TestIndexJoinOnClusteredIndex", + "Cases": [ + { + "SQL": "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", + "Plan": [ + "IndexJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a, equal cond:eq(test.t.a, test.t.a)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", + "└─TableReader(Probe) 3.00 root data:TableRangeScan", + " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a)], keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11 1 111 1.1000000000 11", + "2 222 2.2000000000 12 2 222 2.2000000000 12", + "3 333 3.3000000000 13 3 333 3.3000000000 13" + ] + }, + { + "SQL": "select /*+ inl_merge_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", + "Plan": [ + "IndexMergeJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", + "└─TableReader(Probe) 3.00 root data:TableRangeScan", + " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a)], keep order:true" + ], + "Res": [ + "1 111 1.1000000000 11 1 111 1.1000000000 11", + "2 222 2.2000000000 12 2 222 2.2000000000 12", + "3 333 3.3000000000 13 3 333 3.3000000000 13" + ] + }, + { + "SQL": "select /*+ inl_hash_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", + "Plan": [ + "IndexHashJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a, equal cond:eq(test.t.a, test.t.a)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", + "└─TableReader(Probe) 3.00 root data:TableRangeScan", + " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a)], keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11 1 111 1.1000000000 11", + "2 222 2.2000000000 12 2 222 2.2000000000 12", + "3 333 3.3000000000 13 3 333 3.3000000000 13" + ] + }, + { + "SQL": "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a and t1.b = t2.b", + "Plan": [ + "IndexJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, test.t.b, inner key:test.t.a, test.t.b, equal cond:eq(test.t.a, test.t.a), eq(test.t.b, test.t.b)", + "├─TableReader(Build) 3.00 root data:TableFullScan", + "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", + "└─TableReader(Probe) 3.00 root data:TableRangeScan", + " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a) eq(test.t.b, test.t.b)], keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11 1 111 1.1000000000 11", + "2 222 2.2000000000 12 2 222 2.2000000000 12", + "3 333 3.3000000000 13 3 333 3.3000000000 13" + ] + }, + { + "SQL": "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.c = t2.c", + "Plan": [ + "IndexJoin 3.00 root inner join, inner:IndexLookUp, outer key:test.t.c, inner key:test.t.c, equal cond:eq(test.t.c, test.t.c)", + "├─TableReader(Build) 3.00 root data:Selection", + "│ └─Selection 3.00 cop[tikv] not(isnull(test.t.c))", + "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", + "└─IndexLookUp(Probe) 3.00 root ", + " ├─Selection(Build) 3.00 cop[tikv] not(isnull(test.t.c))", + " │ └─IndexRangeScan 3.00 cop[tikv] table:t2, index:c(c) range: decided by [eq(test.t.c, test.t.c)], keep order:false", + " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t2 keep order:false" + ], + "Res": [ + "1 111 1.1000000000 11 1 111 1.1000000000 11", + "2 222 2.2000000000 12 2 222 2.2000000000 12", + "3 333 3.3000000000 13 3 333 3.3000000000 13" + ] + }, + { + "SQL": "select /*+ inl_merge_join(t1,t2) */ t2.a, t2.c, t2.d from t t1 left join t t2 on t1.a = t2.c;", + "Plan": [ + "IndexMergeJoin 3.00 root left outer join, inner:Projection, outer key:Column#9, inner key:test.t.c", + "├─Projection(Build) 3.00 root cast(test.t.a, decimal(10,0) BINARY)->Column#9", + "│ └─IndexReader 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t1, index:c(c) keep order:false", + "└─Projection(Probe) 3.00 root test.t.a, test.t.c, test.t.d", + " └─IndexLookUp 3.00 root ", + " ├─IndexRangeScan(Build) 3.00 cop[tikv] table:t2, index:c(c) range: decided by [eq(test.t.c, Column#9)], keep order:true", + " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t2 keep order:false" + ], + "Res": [ + " ", + " ", + " " + ] + } + ] + }, + { + "Name": "TestHeuristicIndexSelection", + "Cases": [ + { + "SQL": "select * from t1 where a = 3 or a = 5", + "Plan": [ + "Batch_Point_Get_5 2.00 887.04 root table:t1 handle:[3 5], keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 handle of t1 is selected since the path only has point ranges" + ] + }, + { + "SQL": "select f, g from t1 where f = 2 and g in (3, 4, 5)", + "Plan": [ + "Batch_Point_Get_5 3.00 380.16 root table:t1, index:f_g(f, g) keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 unique index f_g of t1 is selected since the path only has point ranges with single scan" + ] + }, + { + "SQL": "select * from t1 where c = 1 and (d = 2 or d = 3) and e in (4, 5)", + "Plan": [ + "Batch_Point_Get_5 4.00 1774.08 root table:t1, index:c_d_e(c, d, e) keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 unique index c_d_e of t1 is selected since the path only has point ranges with double scan" + ] + }, + { + "SQL": "select f, g from t1 where f = 2 and g > 3", + "Plan": [ + "IndexReader_6 33.33 733.82 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 33.33 6783.33 cop[tikv] table:t1, index:f_g(f, g) range:(2 3,2 +inf], keep order:false, stats:pseudo" + ], + "Warnings": [ + "Note 1105 unique index f_g of t1 is selected since the path only fetches limited number of rows with single scan" + ] + }, + { + "SQL": "select a, b, c from t2 where a = 1 and b = 2 and c in (1, 2, 3, 4, 5)", + "Plan": [ + "Selection_6 0.01 289.88 root eq(test.t2.b, 2), in(test.t2.c, 1, 2, 3, 4, 5)", + "└─Point_Get_5 1.00 190.08 root table:t2, index:idx_a(a) " + ], + "Warnings": [ + "Note 1105 unique index idx_a of t2 is selected since the path only has point ranges with double scan" + ] + }, + { + "SQL": "select * from t3 where (a = 1 or a = 3) and b = 'xx'", + "Plan": [ + "Batch_Point_Get_5 2.00 1449.36 root table:t3, clustered index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 handle of t3 is selected since the path only has point ranges" + ] + }, + { + "SQL": "select * from t4 where (a = 1 or a = 3) and b = 'xx'", + "Plan": [ + "Batch_Point_Get_5 2.00 1449.36 root table:t4, index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 unique index PRIMARY of t4 is selected since the path only has point ranges with double scan" + ] + }, + { + "SQL": "select a, b from t3 where (a = 1 or a = 3) and b = 'xx'", + "Plan": [ + "Batch_Point_Get_5 2.00 1322.64 root table:t3, clustered index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 handle of t3 is selected since the path only has point ranges" + ] + }, + { + "SQL": "select a, b from t4 where (a = 1 or a = 3) and b = 'xx'", + "Plan": [ + "Batch_Point_Get_5 2.00 1322.64 root table:t4, index:PRIMARY(a, b) keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 unique index PRIMARY of t4 is selected since the path only has point ranges with single scan" + ] + }, + { + "SQL": "update t1 set b = 2 where a = 4 or a = 6", + "Plan": [ + "Update_4 N/A N/A root N/A", + "└─Batch_Point_Get_6 2.00 887.04 root table:t1 handle:[4 6], keep order:false, desc:false" + ], + "Warnings": [ + "Note 1105 handle of t1 is selected since the path only has point ranges" + ] + }, + { + "SQL": "delete from t1 where f = 2 and g in (3, 4)", + "Plan": [ + "Delete_4 N/A N/A root N/A", + "└─Selection_7 2.00 493.42 root in(test.t1.g, 3, 4)", + " └─Point_Get_6 1.00 443.52 root table:t1, index:f(f) " + ], + "Warnings": [ + "Note 1105 unique index f of t1 is selected since the path only has point ranges with double scan" + ] + }, + { + "SQL": "insert into t3 select a, b, c from t1 where f = 2", + "Plan": [ + "Insert_1 N/A N/A root N/A", + "└─Projection_6 1.00 253.74 root test.t1.a, test.t1.b, test.t1.c", + " └─Point_Get_7 1.00 253.44 root table:t1, index:f(f) " + ], + "Warnings": [ + "Note 1105 unique index f of t1 is selected since the path only has point ranges with double scan" + ] + }, + { + "SQL": "replace into t3 select a, b, c from t1 where a = 3", + "Plan": [ + "Insert_1 N/A N/A root N/A", + "└─Point_Get_7 1.00 190.08 root table:t1 handle:3" + ], + "Warnings": [ + "Note 1105 handle of t1 is selected since the path only has point ranges" + ] + } + ] + }, + { + "Name": "TestIndexMergeWithCorrelatedColumns", + "Cases": [ + { + "SQL": "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", + "Plan": [ + "Sort 10000.00 root test.t2.c1", + "└─Projection 10000.00 root test.t2.c1, test.t2.c2, test.t2.c3", + " └─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.t2.c1, Column#8), if(ne(Column#9, 0), NULL, 1)), or(eq(Column#10, 0), if(isnull(test.t2.c1), NULL, 0)))", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", + " └─IndexMerge 63.35 root type: union", + " ├─Selection(Build) 10000.00 cop[tikv] eq(10, test.t2.c3)", + " │ └─TableRangeScan 10000.00 cop[tikv] table:t1 range:[10,10], keep order:false, stats:pseudo", + " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", + " │ └─IndexRangeScan 100000.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 63.35 cop[tikv] or(and(eq(test.t1.c1, 10), eq(10, test.t2.c3)), and(eq(test.t1.c2, 1), eq(1, test.t2.c3))), substring(cast(test.t1.c3, var_string(20)), 10)", + " └─TableRowIDScan 89992.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Res": [ + "1 1 1", + "2 2 2" + ] + }, + { + "SQL": "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and reverse(c3)) order by c1;", + "Plan": [ + "Sort 10000.00 root test.t2.c1", + "└─Projection 10000.00 root test.t2.c1, test.t2.c2, test.t2.c3", + " └─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.t2.c1, Column#8), if(ne(Column#9, 0), NULL, 1)), or(eq(Column#10, 0), if(isnull(test.t2.c1), NULL, 0)))", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", + " └─IndexMerge 63.35 root type: union", + " ├─Selection(Build) 10000.00 cop[tikv] eq(10, test.t2.c3)", + " │ └─TableRangeScan 10000.00 cop[tikv] table:t1 range:[10,10], keep order:false, stats:pseudo", + " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", + " │ └─IndexRangeScan 100000.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 63.35 cop[tikv] or(and(eq(test.t1.c1, 10), eq(10, test.t2.c3)), and(eq(test.t1.c2, 1), eq(1, test.t2.c3))), reverse(cast(test.t1.c3, var_string(20)))", + " └─TableRowIDScan 89992.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Res": [ + "2 2 2" + ] + }, + { + "SQL": "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 >= 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", + "Plan": [ + "Sort 10000.00 root test.t2.c1", + "└─Projection 10000.00 root test.t2.c1, test.t2.c2, test.t2.c3", + " └─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.t2.c1, Column#8), if(ne(Column#9, 0), NULL, 1)), or(eq(Column#10, 0), if(isnull(test.t2.c1), NULL, 0)))", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", + " └─IndexMerge 30263.46 root type: union", + " ├─Selection(Build) 33333.33 cop[tikv] eq(test.t1.c1, test.t2.c3)", + " │ └─TableRangeScan 33333333.33 cop[tikv] table:t1 range:[10,+inf], keep order:false, stats:pseudo", + " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", + " │ └─IndexRangeScan 100000.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 30263.46 cop[tikv] or(and(ge(test.t1.c1, 10), eq(test.t1.c1, test.t2.c3)), and(eq(test.t1.c2, 1), eq(1, test.t2.c3))), substring(cast(test.t1.c3, var_string(20)), 10)", + " └─TableRowIDScan 33386666.67 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Res": [ + "1 1 1", + "2 2 2" + ] + }, + { + "SQL": "select c_int from tt1 where c_decimal < all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt1.c_int = tt2.c_int and tt1.c_datetime > tt2.c_datetime and tt2.c_decimal = 9.060 or tt2.c_str <= 'interesting shtern' and tt1.c_int = tt2.c_int) order by 1;", + "Plan": [ + "Projection 10000.00 root test.tt1.c_int", + "└─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.tt1.c_decimal, Column#9), if(ne(Column#10, 0), NULL, 1)), or(eq(Column#11, 0), if(isnull(test.tt1.c_decimal), NULL, 0)))", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:tt1 keep order:true, stats:pseudo", + " └─StreamAgg(Probe) 10000.00 root funcs:min(Column#14)->Column#9, funcs:sum(Column#15)->Column#10, funcs:count(1)->Column#11", + " └─Projection 11.05 root test.tt2.c_decimal->Column#14, cast(isnull(test.tt2.c_decimal), decimal(20,0) BINARY)->Column#15", + " └─IndexMerge 11.05 root type: union", + " ├─Selection(Build) 10.00 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", + " │ └─IndexRangeScan 10000.00 cop[tikv] table:tt2, index:c_decimal(c_decimal) range:[9.060000,9.060000], keep order:false, stats:pseudo", + " ├─Selection(Build) 33233.33 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", + " │ └─IndexRangeScan 33233333.33 cop[tikv] table:tt2, index:c_str(c_str) range:[-inf,\"interesting shtern\"], keep order:false, stats:pseudo", + " └─Selection(Probe) 11.05 cop[tikv] or(and(eq(test.tt1.c_int, test.tt2.c_int), and(gt(test.tt1.c_datetime, test.tt2.c_datetime), eq(test.tt2.c_decimal, 9.060))), and(le(test.tt2.c_str, \"interesting shtern\"), eq(test.tt1.c_int, test.tt2.c_int)))", + " └─TableRowIDScan 33243.33 cop[tikv] table:tt2 keep order:false, stats:pseudo" + ], + "Res": [ + "7", + "8", + "10" + ] + }, + { + "SQL": "select c_int from tt1 where c_decimal > all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt2.c_int = 7 and tt2.c_int < tt1.c_decimal or tt2.c_str >= 'zzzzzzzzzzzzzzzzzzz' and tt1.c_int = tt2.c_int) order by 1;", + "Plan": [ + "Projection 10000.00 root test.tt1.c_int", + "└─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(gt(test.tt1.c_decimal, Column#9), if(ne(Column#10, 0), NULL, 1)), or(eq(Column#11, 0), if(isnull(test.tt1.c_decimal), NULL, 0)))", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:tt1 keep order:true, stats:pseudo", + " └─StreamAgg(Probe) 10000.00 root funcs:max(Column#14)->Column#9, funcs:sum(Column#15)->Column#10, funcs:count(1)->Column#11", + " └─Projection 17.91 root test.tt2.c_decimal->Column#14, cast(isnull(test.tt2.c_decimal), decimal(20,0) BINARY)->Column#15", + " └─IndexMerge 17.91 root type: union", + " ├─Selection(Build) 10000.00 cop[tikv] lt(7, test.tt1.c_decimal)", + " │ └─TableRangeScan 10000.00 cop[tikv] table:tt2 range:[7,7], keep order:false, stats:pseudo", + " ├─Selection(Build) 33333.33 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", + " │ └─IndexRangeScan 33333333.33 cop[tikv] table:tt2, index:c_str(c_str) range:[\"zzzzzzzzzzzzzzzzzzz\",+inf], keep order:false, stats:pseudo", + " └─Selection(Probe) 17.91 cop[tikv] or(and(eq(test.tt2.c_int, 7), lt(7, test.tt1.c_decimal)), and(ge(test.tt2.c_str, \"zzzzzzzzzzzzzzzzzzz\"), eq(test.tt1.c_int, test.tt2.c_int)))", + " └─TableRowIDScan 43330.00 cop[tikv] table:tt2 keep order:false, stats:pseudo" + ], + "Res": [ + "6", + "7", + "8", + "9" + ] + } + ] + }, + { + "Name": "TestIndexMergeSerial", + "Cases": [ + { + "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", + "Plan": [ + "IndexMerge 8.00 root type: union", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─Selection(Build) 1.00 cop[tikv] 1", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" + ], + "Warnings": null + }, + { + "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", + "Plan": [ + "IndexMerge 8.00 root type: union", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─Selection(Build) 1.00 cop[tikv] 1", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" + ], + "Warnings": null + }, + { + "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", + "Plan": [ + "IndexMerge 8.00 root type: union", + "├─Selection(Build) 1.00 cop[tikv] 1", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─Selection(Build) 1.00 cop[tikv] 1", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" + ], + "Warnings": null + }, + { + "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)", + "Plan": [ + "IndexMerge 0.29 root type: union", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─Selection(Probe) 0.29 cop[tikv] or(and(eq(test.t.a, 1), eq(length(cast(test.t.b, var_string(20))), 1)), and(eq(test.t.b, 1), eq(length(cast(test.t.a, var_string(20))), 1)))", + " └─TableRowIDScan 1.90 cop[tikv] table:t keep order:false" + ], + "Warnings": null + } + ] + }, + { + "Name": "TestLimitIndexLookUpKeepOrder", + "Cases": [ + { + "SQL": "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", + "Plan": [ + "Limit 0.00 root offset:0, count:10", + "└─Projection 0.00 root test.t.a, test.t.b, test.t.c, test.t.d", + " └─IndexLookUp 0.00 root ", + " ├─IndexRangeScan(Build) 2.50 cop[tikv] table:t, index:idx(a, b, c) range:(1 2,1 10), keep order:true, stats:pseudo", + " └─Selection(Probe) 0.00 cop[tikv] eq(test.t.d, 10)", + " └─TableRowIDScan 2.50 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10", + "Plan": [ + "Limit 0.00 root offset:0, count:10", + "└─Projection 0.00 root test.t.a, test.t.b, test.t.c, test.t.d", + " └─IndexLookUp 0.00 root ", + " ├─IndexRangeScan(Build) 2.50 cop[tikv] table:t, index:idx(a, b, c) range:(1 2,1 10), keep order:true, desc, stats:pseudo", + " └─Selection(Probe) 0.00 cop[tikv] eq(test.t.d, 10)", + " └─TableRowIDScan 2.50 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIndexJoinRangeFallback", + "Cases": [ + { + "SQL": "set @@tidb_opt_range_max_size = 0", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "Plan": [ + "IndexJoin 0.50 root inner join, inner:IndexLookUp, outer key:test.t2.e, test.t2.g, inner key:test.t1.b, test.t1.d, equal cond:eq(test.t2.e, test.t1.b), eq(test.t2.g, test.t1.d)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 0.50 root ", + " ├─Selection(Build) 249.50 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", + " │ └─IndexRangeScan 250.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.b, test.t2.e) eq(test.t1.d, test.t2.g) in(test.t1.a, 1, 3) in(test.t1.c, aaa, bbb)], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.50 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", + " └─TableRowIDScan 249.50 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "set @@tidb_opt_range_max_size = 2900", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "Plan": [ + "IndexJoin 0.50 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.b, equal cond:eq(test.t2.e, test.t1.b), eq(test.t2.g, test.t1.d)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 0.50 root ", + " ├─Selection(Build) 249.50 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", + " │ └─IndexRangeScan 250.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.b, test.t2.e) in(test.t1.a, 1, 3) in(test.t1.c, aaa, bbb)], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.50 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", + " └─TableRowIDScan 249.50 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "Memory capacity of 2900 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" + ] + }, + { + "SQL": "set @@tidb_opt_range_max_size = 2300", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "Plan": [ + "IndexJoin 0.50 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.b, equal cond:eq(test.t2.e, test.t1.b), eq(test.t2.g, test.t1.d)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 0.50 root ", + " ├─Selection(Build) 249.50 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", + " │ └─IndexRangeScan 250.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.b, test.t2.e) in(test.t1.a, 1, 3)], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.50 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", + " └─TableRowIDScan 249.50 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "Memory capacity of 2300 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" + ] + }, + { + "SQL": "set @@tidb_opt_range_max_size = 700", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", + "Plan": [ + "HashJoin 0.05 root inner join, equal:[eq(test.t1.b, test.t2.e) eq(test.t1.d, test.t2.g)]", + "├─IndexLookUp(Build) 0.04 root ", + "│ ├─Selection(Build) 19.96 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", + "│ │ └─IndexRangeScan 20.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range:[1,1], [3,3], keep order:false, stats:pseudo", + "│ └─Selection(Probe) 0.04 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", + "│ └─TableRowIDScan 19.96 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "Memory capacity of 700 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", + "[planner:1815]Optimizer Hint /*+ INL_JOIN(t1) */ or /*+ TIDB_INLJ(t1) */ is inapplicable" + ] + }, + { + "SQL": "set @@tidb_opt_range_max_size = 0", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", + "Plan": [ + "IndexJoin 312.19 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.e))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 312.19 root ", + " ├─Selection(Build) 312.19 cop[tikv] not(isnull(test.t1.a))", + " │ └─IndexRangeScan 312.50 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e) gt(test.t1.b, 1) lt(test.t1.b, 10)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 312.19 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "set @@tidb_opt_range_max_size = 300", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", + "Plan": [ + "IndexJoin 312.19 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.e))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 312.19 root ", + " ├─Selection(Build) 312.19 cop[tikv] gt(test.t1.b, 1), lt(test.t1.b, 10), not(isnull(test.t1.a))", + " │ └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 312.19 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "Memory capacity of 300 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" + ] + }, + { + "SQL": "set @@tidb_opt_range_max_size = 0", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10", + "Plan": [ + "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a), other cond:gt(test.t1.b, test.t2.f), lt(test.t1.b, plus(test.t2.f, 10))", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.f))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 12475.01 root ", + " ├─Selection(Build) 12475.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e) lt(test.t1.b, plus(test.t2.f, 10)) gt(test.t1.b, test.t2.f)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "set @@tidb_opt_range_max_size = 300", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10", + "Plan": [ + "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a), other cond:gt(test.t1.b, test.t2.f), lt(test.t1.b, plus(test.t2.f, 10))", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.f))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 12475.01 root ", + " ├─Selection(Build) 12475.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "Memory capacity of 300 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" + ] + } + ] + }, + { + "Name": "TestNullConditionForPrefixIndex", + "Cases": [ + { + "SQL": "select count(1) from t1 where c1 = '0xfff' and c2 is not null", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#7)->Column#5", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo" + ], + "Result": [ + "3" + ] + }, + { + "SQL": "select count(1) from t1 where c1 = '0xfff' and c2 is null", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#5", + "└─IndexReader 0.10 root index:IndexRangeScan", + " └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" NULL,\"0xfff\" NULL], keep order:false, stats:pseudo" + ], + "Result": [ + "1" + ] + }, + { + "SQL": "select count(1) from t1 where c1 >= '0xfff' and c2 is not null", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#6)->Column#5", + "└─IndexReader 1.00 root index:HashAgg", + " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#6", + " └─Selection 3330.00 cop[tikv] not(isnull(test.t1.c2))", + " └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "3" + ] + }, + { + "SQL": "select count(1) from t1 where c1 >= '0xfff' and c2 is null", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#7)->Column#5", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#7", + " └─Selection 3.33 cop[tikv] isnull(test.t1.c2)", + " └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "1" + ] + }, + { + "SQL": "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is not null", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#5", + "└─IndexLookUp 8.00 root ", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:idx1(c1) range:[\"0xfff\",\"0xfff\"], keep order:false, stats:pseudo", + " └─Selection(Probe) 8.00 cop[tikv] not(isnull(plus(cast(test.t1.c2, double BINARY), 1)))", + " └─TableRowIDScan 10.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Result": [ + "3" + ] + }, + { + "SQL": "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is null", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#5", + "└─IndexLookUp 8.00 root ", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:idx1(c1) range:[\"0xfff\",\"0xfff\"], keep order:false, stats:pseudo", + " └─Selection(Probe) 8.00 cop[tikv] isnull(plus(cast(test.t1.c2, double BINARY), 1))", + " └─TableRowIDScan 10.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Result": [ + "1" + ] + }, + { + "SQL": "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is not null", + "Plan": [ + "Projection 99.90 root test.t1.c2", + "└─IndexLookUp 99.90 root ", + " ├─IndexRangeScan(Build) 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 99.90 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Result": [ + "", + "111111", + "22 " + ] + }, + { + "SQL": "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is null", + "Plan": [ + "Projection 0.10 root test.t1.c2", + "└─IndexLookUp 0.10 root ", + " ├─IndexRangeScan(Build) 0.10 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" NULL,\"0xfff\" NULL], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.10 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Result": [ + "" + ] + }, + { + "SQL": "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is not null", + "Plan": [ + "Projection 3330.00 root test.t1.c2", + "└─IndexLookUp 3330.00 root ", + " ├─Selection(Build) 3330.00 cop[tikv] not(isnull(test.t1.c2))", + " │ └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 3330.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Result": [ + "", + "111111", + "22 " + ] + }, + { + "SQL": "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is null", + "Plan": [ + "Projection 3.33 root test.t1.c2", + "└─IndexLookUp 3.33 root ", + " ├─Selection(Build) 3.33 cop[tikv] isnull(test.t1.c2)", + " │ └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 3.33 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Result": [ + "" + ] + }, + { + "SQL": "select count(1) from t2 use index(idx) where b is not null", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#5)->Column#4", + "└─IndexReader 1.00 root index:HashAgg", + " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#5", + " └─IndexFullScan 9990.00 cop[tikv] table:t2, index:idx(b) keep order:false, stats:pseudo" + ], + "Result": [ + "3" + ] + }, + { + "SQL": "select count(1) from t2 use index(idx) where b is null", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#6)->Column#4", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan 10.00 cop[tikv] table:t2, index:idx(b) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + "1" + ] + }, + { + "SQL": "select b from t2 use index(idx) where b is not null", + "Plan": [ + "IndexLookUp 9990.00 root ", + "├─IndexFullScan(Build) 9990.00 cop[tikv] table:t2, index:idx(b) keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 9990.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Result": [ + "", + "aaaaaa", + "bb " + ] + }, + { + "SQL": "select b from t2 use index(idx) where b is null", + "Plan": [ + "IndexLookUp 10.00 root ", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:idx(b) range:[NULL,NULL], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Result": [ + "" + ] + }, + { + "SQL": "select b from t3 where a = 1 and b is not null", + "Plan": [ + "Projection 10.00 root test.t3.b", + "└─TableReader 10.00 root data:TableRangeScan", + " └─TableRangeScan 10.00 cop[tikv] table:t3 range:[1,1], keep order:false, stats:pseudo" + ], + "Result": [ + "", + "aaaaaa", + "bb " + ] + }, + { + "SQL": "select b from t3 where a = 1 and b is null", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Result": null + } + ] + } +] diff --git a/planner/core/casetest/integration_test.go b/planner/core/casetest/integration_test.go index ada8f1cdd5bcf..a0b74f212a524 100644 --- a/planner/core/casetest/integration_test.go +++ b/planner/core/casetest/integration_test.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" - "github.com/pingcap/tidb/util" "github.com/stretchr/testify/require" ) @@ -305,88 +304,6 @@ func TestMaxMinEliminate(t *testing.T) { } } -func TestIndexJoinUniqueCompositeIndex(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly - tk.MustExec("create table t1(a int not null, c int not null)") - tk.MustExec("create table t2(a int not null, b int not null, c int not null, primary key(a,b))") - tk.MustExec("insert into t1 values(1,1)") - tk.MustExec("insert into t2 values(1,1,1),(1,2,1)") - tk.MustExec("analyze table t1,t2") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestIndexMerge(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int, c int, unique index(a), unique index(b), primary key(c))") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -// for issue #14822 and #38258 -func TestIndexJoinTableRange(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(a int, b int, primary key (a), key idx_t1_b (b))") - tk.MustExec("create table t2(a int, b int, primary key (a), key idx_t1_b (b))") - tk.MustExec("create table t3(a int, b int, c int)") - tk.MustExec("create table t4(a int, b int, c int, primary key (a, b) clustered)") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestSubqueryWithTopN(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -440,63 +357,6 @@ func TestApproxPercentile(t *testing.T) { } } -func TestIndexJoinInnerIndexNDV(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(a int not null, b int not null, c int not null)") - tk.MustExec("create table t2(a int not null, b int not null, c int not null, index idx1(a,b), index idx2(c))") - tk.MustExec("insert into t1 values(1,1,1),(1,1,1),(1,1,1)") - tk.MustExec("insert into t2 values(1,1,1),(1,1,2),(1,1,3)") - tk.MustExec("analyze table t1, t2") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func TestIndexMergeSerial(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a int, b int, unique key(a), unique key(b))") - tk.MustExec("insert into t value (1, 5), (2, 4), (3, 3), (4, 2), (5, 1)") - tk.MustExec("insert into t value (6, 0), (7, -1), (8, -2), (9, -3), (10, -4)") - tk.MustExec("analyze table t") - - var input []string - var output []struct { - SQL string - Plan []string - Warnings []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) - } -} - func TestStreamAggProp(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -525,36 +385,6 @@ func TestStreamAggProp(t *testing.T) { } } -func TestIndexJoinOnClusteredIndex(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("drop table if exists t1") - tk.MustExec("create table t (a int, b varchar(20), c decimal(40,10), d int, primary key(a,b), key(c))") - tk.MustExec(`insert into t values (1,"111",1.1,11), (2,"222",2.2,12), (3,"333",3.3,13)`) - tk.MustExec("analyze table t") - - var input []string - var output []struct { - SQL string - Plan []string - Res []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery("explain format = 'brief'" + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) - } -} - func TestPartitionExplain(t *testing.T) { failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") @@ -736,30 +566,6 @@ func TestMergeContinuousSelections(t *testing.T) { } } -func TestLimitIndexLookUpKeepOrder(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a int, b int, c int, d int, index idx(a,b,c));") - - var input []string - var output []struct { - SQL string - Plan []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - func TestDecorrelateInnerJoinInSubquery(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -1040,39 +846,6 @@ func TestSequenceAsDataSource(t *testing.T) { } } -func TestHeuristicIndexSelection(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(a int, b int, c int, d int, e int, f int, g int, primary key (a), unique key c_d_e (c, d, e), unique key f (f), unique key f_g (f, g), key g (g))") - tk.MustExec("create table t2(a int, b int, c int, d int, unique index idx_a (a), unique index idx_b_c (b, c), unique index idx_b_c_a_d (b, c, a, d))") - tk.MustExec("create table t3(a bigint, b varchar(255), c bigint, primary key(a, b) clustered)") - tk.MustExec("create table t4(a bigint, b varchar(255), c bigint, primary key(a, b) nonclustered)") - - // Default RPC encoding may cause statistics explain result differ and then the test unstable. - tk.MustExec("set @@tidb_enable_chunk_rpc = on") - - var input []string - var output []struct { - SQL string - Plan []string - Warnings []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'verbose' " + tt).Rows()) - output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) - }) - tk.MustQuery("explain format = 'verbose' " + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) - } -} - func TestOutputSkylinePruningInfo(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -1201,53 +974,6 @@ func TestGroupBySetVar(t *testing.T) { } } -func TestIndexMergeWithCorrelatedColumns(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test;") - - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("drop table if exists t1, t2;") - tk.MustExec("create table t1(c1 int, c2 int, c3 int, primary key(c1), key(c2));") - tk.MustExec("insert into t1 values(1, 1, 1);") - tk.MustExec("insert into t1 values(2, 2, 2);") - tk.MustExec("create table t2(c1 int, c2 int, c3 int);") - tk.MustExec("insert into t2 values(1, 1, 1);") - tk.MustExec("insert into t2 values(2, 2, 2);") - - tk.MustExec("drop table if exists tt1, tt2;") - tk.MustExec("create table tt1 (c_int int, c_str varchar(40), c_datetime datetime, c_decimal decimal(12, 6), primary key(c_int), key(c_int), key(c_str), unique key(c_decimal), key(c_datetime));") - tk.MustExec("create table tt2 like tt1 ;") - tk.MustExec(`insert into tt1 (c_int, c_str, c_datetime, c_decimal) values (6, 'sharp payne', '2020-06-07 10:40:39', 6.117000) , - (7, 'objective kare', '2020-02-05 18:47:26', 1.053000) , - (8, 'thirsty pasteur', '2020-01-02 13:06:56', 2.506000) , - (9, 'blissful wilbur', '2020-06-04 11:34:04', 9.144000) , - (10, 'reverent mclean', '2020-02-12 07:36:26', 7.751000) ;`) - tk.MustExec(`insert into tt2 (c_int, c_str, c_datetime, c_decimal) values (6, 'beautiful joliot', '2020-01-16 01:44:37', 5.627000) , - (7, 'hopeful blackburn', '2020-05-23 21:44:20', 7.890000) , - (8, 'ecstatic davinci', '2020-02-01 12:27:17', 5.648000) , - (9, 'hopeful lewin', '2020-05-05 05:58:25', 7.288000) , - (10, 'sharp jennings', '2020-01-28 04:35:03', 9.758000) ;`) - - var input []string - var output []struct { - SQL string - Plan []string - Res []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format=brief " + tt).Rows()) - output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery("explain format=brief " + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) - } -} - func TestIssue31240(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) @@ -1462,62 +1188,6 @@ func TestDowncastPointGetOrRangeScan(t *testing.T) { } } -func TestNullConditionForPrefixIndex(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`CREATE TABLE t1 ( - id char(1) DEFAULT NULL, - c1 varchar(255) DEFAULT NULL, - c2 text DEFAULT NULL, - KEY idx1 (c1), - KEY idx2 (c1,c2(5)) -) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin`) - tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("create table t2(a int, b varchar(10), index idx(b(5)))") - tk.MustExec("create table t3(a int, b varchar(10), c int, primary key (a, b(5)) clustered)") - tk.MustExec("set tidb_opt_prefix_index_single_scan = 1") - tk.MustExec("insert into t1 values ('a', '0xfff', '111111'), ('b', '0xfff', '22 '), ('c', '0xfff', ''), ('d', '0xfff', null)") - tk.MustExec("insert into t2 values (1, 'aaaaaa'), (2, 'bb '), (3, ''), (4, null)") - tk.MustExec("insert into t3 values (1, 'aaaaaa', 2), (1, 'bb ', 3), (1, '', 4)") - - var input []string - var output []struct { - SQL string - Plan []string - Result []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - testdata.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + tt).Rows()) - output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) - }) - tk.MustQuery("explain format='brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Result...)) - } - - // test plan cache - tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) - tk.MustExec("set @@tidb_enable_collect_execution_info=0") - tk.MustExec("prepare stmt from 'select count(1) from t1 where c1 = ? and c2 is not null'") - tk.MustExec("set @a = '0xfff'") - tk.MustQuery("execute stmt using @a").Check(testkit.Rows("3")) - tk.MustQuery("execute stmt using @a").Check(testkit.Rows("3")) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) - tk.MustQuery("execute stmt using @a").Check(testkit.Rows("3")) - tkProcess := tk.Session().ShowProcess() - ps := []*util.ProcessInfo{tkProcess} - tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) - tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( - "StreamAgg_17 1.00 root funcs:count(Column#7)->Column#5", - "└─IndexReader_18 1.00 root index:StreamAgg_9", - " └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#7", - " └─IndexRangeScan_16 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo")) -} - // https://github.com/pingcap/tidb/issues/24095 func TestIssue24095(t *testing.T) { store := testkit.CreateMockStore(t) @@ -1544,40 +1214,6 @@ func TestIssue24095(t *testing.T) { } } -func TestIndexJoinRangeFallback(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(a int, b int, c varchar(10), d varchar(10), index idx_a_b_c_d(a, b, c(2), d(2)))") - tk.MustExec("create table t2(e int, f int, g varchar(10), h varchar(10))") - - var input []string - var output []struct { - SQL string - Plan []string - Warn []string - } - integrationSuiteData := GetIntegrationSuiteData() - integrationSuiteData.LoadTestCases(t, &input, &output) - for i, tt := range input { - setStmt := strings.HasPrefix(tt, "set") - testdata.OnRecord(func() { - output[i].SQL = tt - if !setStmt { - output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) - } - }) - if setStmt { - tk.MustExec(tt) - } else { - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) - } - } -} - func TestFixControl45132(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/planner/core/casetest/pushdown/BUILD.bazel b/planner/core/casetest/pushdown/BUILD.bazel index 372c576a3fda0..87d76e24b2ffa 100644 --- a/planner/core/casetest/pushdown/BUILD.bazel +++ b/planner/core/casetest/pushdown/BUILD.bazel @@ -9,7 +9,6 @@ go_test( ], data = glob(["testdata/**"]), flaky = True, - race = "on", shard_count = 8, deps = [ "//domain", diff --git a/planner/core/casetest/pushdown/push_down_test.go b/planner/core/casetest/pushdown/push_down_test.go index 6958c5cff8b7d..b2bb0ddb57e50 100644 --- a/planner/core/casetest/pushdown/push_down_test.go +++ b/planner/core/casetest/pushdown/push_down_test.go @@ -27,6 +27,18 @@ import ( func TestPushLimitDownIndexLookUpReader(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) + tk.MustExec("create database testa") + tk.MustExec("use testa") + tk.MustExec("drop table if exists tbl") + tk.MustExec("create table tbl(a int, b int, c int, key idx_b_c(b,c))") + tk.MustExec("insert into tbl values(1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5)") + tk.MustQuery("select * from tbl use index(idx_b_c) where b > 1 limit 2,1").Check(testkit.Rows("4 4 4")) + tk.MustQuery("select * from tbl use index(idx_b_c) where b > 4 limit 2,1").Check(testkit.Rows()) + tk.MustQuery("select * from tbl use index(idx_b_c) where b > 3 limit 2,1").Check(testkit.Rows()) + tk.MustQuery("select * from tbl use index(idx_b_c) where b > 2 limit 2,1").Check(testkit.Rows("5 5 5")) + tk.MustQuery("select * from tbl use index(idx_b_c) where b > 1 limit 1").Check(testkit.Rows("2 2 2")) + tk.MustQuery("select * from tbl use index(idx_b_c) where b > 1 order by b desc limit 2,1").Check(testkit.Rows("3 3 3")) + tk.MustQuery("select * from tbl use index(idx_b_c) where b > 1 and c > 1 limit 2,1").Check(testkit.Rows("4 4 4")) tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set @@session.tidb_executor_concurrency = 4;") diff --git a/planner/core/casetest/testdata/integration_suite_in.json b/planner/core/casetest/testdata/integration_suite_in.json index 9a4d4801a4b73..b5099d4e2a2f5 100644 --- a/planner/core/casetest/testdata/integration_suite_in.json +++ b/planner/core/casetest/testdata/integration_suite_in.json @@ -1,17 +1,4 @@ [ - { - "name": "TestPushLimitDownIndexLookUpReader", - "cases": [ - // Limit should be pushed down into IndexLookUpReader, row count of IndexLookUpReader and TableScan should be 1.00. - "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 limit 2,1", - // Projection atop IndexLookUpReader, Limit should be pushed down into IndexLookUpReader, and Projection should have row count 1.00 as well. - "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 order by b desc limit 2,1", - // Limit should be pushed down into IndexLookUpReader when Selection on top of IndexScan. - "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 and c > 1 limit 2,1", - // Limit should NOT be pushed down into IndexLookUpReader when Selection on top of TableScan. - "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 and a > 1 limit 2,1" - ] - }, { "name": "TestIsFromUnixtimeNullRejective", "cases": [ @@ -35,13 +22,6 @@ "SELECT avg(2) FROM(SELECT min(c) FROM t JOIN(SELECT 1 c) d ORDER BY a) e" ] }, - { - "name": "TestIndexJoinInnerIndexNDV", - "cases": [ - // t2 should use idx2 instead of idx1, since idx2 has larger NDV. - "explain format = 'brief' select /*+ inl_join(t2) */ * from t1, t2 where t1.a = t2.a and t1.b = t2.b and t1.c = t2.c" - ] - }, { "name": "TestSimplifyOuterJoinWithCast", "cases": [ @@ -59,17 +39,6 @@ "explain format = 'brief' select min(b), max(b) from cluster_index_t where b = 1" ] }, - { - "name": "TestIndexJoinUniqueCompositeIndex", - "cases": [ - // Row count of IndexScan should be 2. - "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c = t2.c", - // Row count of IndexScan should be 2. - "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c <= t2.b", - // Row count of IndexScan should be 1. - "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t2.b = 1" - ] - }, { "name": "TestPartitionTableStats", "cases": [ @@ -79,15 +48,6 @@ "select * from t order by a limit 3" ] }, - { - "name": "TestIndexMerge", - "cases": [ - "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 1 or b = 2", - "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, A, B) */ * from t where a = 1 or b = 2", - "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, primary) */ * from t where 1 or t.c", - "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b, c) */ * from t where 1 or t.a = 1 or t.b = 2" - ] - }, { "name": "TestSubqueryWithTopN", "cases": [ @@ -96,15 +56,6 @@ "desc format = 'brief' select * from (select b+b as x from t) t1, t t2 where t1.x=t2.b order by t1.x limit 1" ] }, - { - "name": "TestIndexJoinTableRange", - "cases": [ - "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.b", - "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.a and t1.b = t2.b", - "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.a = t4.a where t4.b = 1", - "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.b = t4.b where t4.a = 1" - ] - }, { "name": "TestPartitionPruningForInExpr", "cases": [ @@ -126,33 +77,6 @@ "select /*+ stream_agg() */ count(*) c from t group by a order by a" ] }, - { - "name": "TestAccessPathOnClusterIndex", - "cases": [ - "select * from t1", - "select * from t1 where t1.a >= 1 and t1.a < 4", - "select * from t1 where t1.a = 1 and t1.b < \"333\"", - "select t1.a, t1.b, t1.c from t1 where t1.c = 3.3", - "select t1.b, t1.c from t1 where t1.c = 2.2", - "select /*+ use_index(t1, c) */ * from t1", - "select * from t1 use index(c) where t1.c in (2.2, 3.3)", - "select * from t1 where t1.a = 1 order by b", - "select * from t1 order by a, b limit 1", - "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a >= 1 or t1.c = 2.2", - "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a = 1 and t1.b = '111' or t1.c = 3.3" - ] - }, - { - "name": "TestIndexJoinOnClusteredIndex", - "cases": [ - "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", - "select /*+ inl_merge_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", - "select /*+ inl_hash_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", - "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a and t1.b = t2.b", - "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.c = t2.c", - "select /*+ inl_merge_join(t1,t2) */ t2.a, t2.c, t2.d from t t1 left join t t2 on t1.a = t2.c;" - ] - }, { "name": "TestPartitionExplain", "cases": [ @@ -267,24 +191,6 @@ "select * from t2 where ((a = 1 and b = 1 and d < 3) or (a = 1 and b = 1 and d > 6)) and c = 3 order by d" ] }, - { - "name": "TestHeuristicIndexSelection", - "cases": [ - "select * from t1 where a = 3 or a = 5", - "select f, g from t1 where f = 2 and g in (3, 4, 5)", - "select * from t1 where c = 1 and (d = 2 or d = 3) and e in (4, 5)", - "select f, g from t1 where f = 2 and g > 3", - "select a, b, c from t2 where a = 1 and b = 2 and c in (1, 2, 3, 4, 5)", - "select * from t3 where (a = 1 or a = 3) and b = 'xx'", - "select * from t4 where (a = 1 or a = 3) and b = 'xx'", - "select a, b from t3 where (a = 1 or a = 3) and b = 'xx'", - "select a, b from t4 where (a = 1 or a = 3) and b = 'xx'", - "update t1 set b = 2 where a = 4 or a = 6", - "delete from t1 where f = 2 and g in (3, 4)", - "insert into t3 select a, b, c from t1 where f = 2", - "replace into t3 select a, b, c from t1 where a = 3" - ] - }, { "name": "TestOutputSkylinePruningInfo", "cases": [ @@ -349,18 +255,6 @@ "select 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab';" ] }, - { - "name": "TestIndexMergeWithCorrelatedColumns", - "cases": [ - "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", - "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and reverse(c3)) order by c1;", - "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 >= 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", - // Test correlated column in IndexPath.TableFilters. - "select c_int from tt1 where c_decimal < all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt1.c_int = tt2.c_int and tt1.c_datetime > tt2.c_datetime and tt2.c_decimal = 9.060 or tt2.c_str <= 'interesting shtern' and tt1.c_int = tt2.c_int) order by 1;", - // Test correlated column in TablePath.TableFilters. - "select c_int from tt1 where c_decimal > all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt2.c_int = 7 and tt2.c_int < tt1.c_decimal or tt2.c_str >= 'zzzzzzzzzzzzzzzzzzz' and tt1.c_int = tt2.c_int) order by 1;" - ] - }, { "name": "TestIssue31240", "cases": [ @@ -434,22 +328,6 @@ "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.a = s.b" ] }, - { - "name": "TestIndexMergeSerial", - "cases": [ - "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", - "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", - "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", - "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)" - ] - }, - { - "name": "TestLimitIndexLookUpKeepOrder", - "cases": [ - "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", - "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10" - ] - }, { "name": "TestIssue23887", "cases": [ @@ -553,48 +431,6 @@ "explain format = 'brief' select row_number() over w1, count(c2) from t1 group by c1 having c1 > 10 window w1 as (partition by c1 order by c2);" ] }, - { - "name": "TestIndexJoinRangeFallback", - "cases": [ - "set @@tidb_opt_range_max_size = 0", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "set @@tidb_opt_range_max_size = 2900", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "set @@tidb_opt_range_max_size = 2300", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "set @@tidb_opt_range_max_size = 700", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "set @@tidb_opt_range_max_size = 0", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", - "set @@tidb_opt_range_max_size = 300", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", - "set @@tidb_opt_range_max_size = 0", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10", - "set @@tidb_opt_range_max_size = 300", - "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10" - ] - }, - { - "name": "TestNullConditionForPrefixIndex", - "cases": [ - "select count(1) from t1 where c1 = '0xfff' and c2 is not null", - "select count(1) from t1 where c1 = '0xfff' and c2 is null", - "select count(1) from t1 where c1 >= '0xfff' and c2 is not null", - "select count(1) from t1 where c1 >= '0xfff' and c2 is null", - "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is not null", - "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is null", - "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is not null", - "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is null", - "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is not null", - "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is null", - "select count(1) from t2 use index(idx) where b is not null", - "select count(1) from t2 use index(idx) where b is null", - "select b from t2 use index(idx) where b is not null", - "select b from t2 use index(idx) where b is null", - "select b from t3 where a = 1 and b is not null", - "select b from t3 where a = 1 and b is null" - ] - }, { "name": "TestDowncastPointGetOrRangeScan", "cases": [ diff --git a/planner/core/casetest/testdata/integration_suite_out.json b/planner/core/casetest/testdata/integration_suite_out.json index c40e12aa02038..0e122dba8cb9c 100644 --- a/planner/core/casetest/testdata/integration_suite_out.json +++ b/planner/core/casetest/testdata/integration_suite_out.json @@ -1,49 +1,4 @@ [ - { - "Name": "TestPushLimitDownIndexLookUpReader", - "Cases": [ - { - "SQL": "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 limit 2,1", - "Plan": [ - "IndexLookUp 1.00 root limit embedded(offset:2, count:1)", - "├─Limit(Build) 3.00 cop[tikv] offset:0, count:3", - "│ └─IndexRangeScan 3.00 cop[tikv] table:tbl, index:idx_b_c(b, c) range:(1,+inf], keep order:false", - "└─TableRowIDScan(Probe) 1.00 cop[tikv] table:tbl keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 order by b desc limit 2,1", - "Plan": [ - "Projection 1.00 root test.tbl.a, test.tbl.b, test.tbl.c", - "└─IndexLookUp 1.00 root limit embedded(offset:2, count:1)", - " ├─Limit(Build) 3.00 cop[tikv] offset:0, count:3", - " │ └─IndexRangeScan 3.00 cop[tikv] table:tbl, index:idx_b_c(b, c) range:(1,+inf], keep order:true, desc", - " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:tbl keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 and c > 1 limit 2,1", - "Plan": [ - "IndexLookUp 1.00 root limit embedded(offset:2, count:1)", - "├─Limit(Build) 3.00 cop[tikv] offset:0, count:3", - "│ └─Selection 3.00 cop[tikv] gt(test.tbl.c, 1)", - "│ └─IndexRangeScan 3.75 cop[tikv] table:tbl, index:idx_b_c(b, c) range:(1,+inf], keep order:false", - "└─TableRowIDScan(Probe) 1.00 cop[tikv] table:tbl keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 and a > 1 limit 2,1", - "Plan": [ - "Limit 1.00 root offset:2, count:1", - "└─IndexLookUp 3.00 root ", - " ├─IndexRangeScan(Build) 3.75 cop[tikv] table:tbl, index:idx_b_c(b, c) range:(1,+inf], keep order:false", - " └─Limit(Probe) 3.00 cop[tikv] offset:0, count:3", - " └─Selection 3.00 cop[tikv] gt(test.tbl.a, 1)", - " └─TableRowIDScan 3.75 cop[tikv] table:tbl keep order:false" - ] - } - ] - }, { "Name": "TestIsFromUnixtimeNullRejective", "Cases": [ @@ -134,22 +89,6 @@ } ] }, - { - "Name": "TestIndexJoinInnerIndexNDV", - "Cases": [ - { - "SQL": "explain format = 'brief' select /*+ inl_join(t2) */ * from t1, t2 where t1.a = t2.a and t1.b = t2.b and t1.c = t2.c", - "Plan": [ - "IndexJoin 3.00 root inner join, inner:IndexLookUp, outer key:test.t1.c, inner key:test.t2.c, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.b, test.t2.b), eq(test.t1.c, test.t2.c)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", - "└─IndexLookUp(Probe) 3.00 root ", - " ├─IndexRangeScan(Build) 3.00 cop[tikv] table:t2, index:idx2(c) range: decided by [eq(test.t2.c, test.t1.c)], keep order:false", - " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t2 keep order:false" - ] - } - ] - }, { "Name": "TestSimplifyOuterJoinWithCast", "Cases": [ @@ -240,44 +179,6 @@ } ] }, - { - "Name": "TestIndexJoinUniqueCompositeIndex", - "Cases": [ - { - "SQL": "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c = t2.c", - "Plan": [ - "IndexJoin 2.00 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.c, test.t2.c)", - "├─TableReader(Build) 1.00 root data:TableFullScan", - "│ └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false", - "└─IndexLookUp(Probe) 2.00 root ", - " ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(test.t2.a, test.t1.a)], keep order:false", - " └─TableRowIDScan(Probe) 2.00 cop[tikv] table:t2 keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t1.c <= t2.b", - "Plan": [ - "IndexJoin 2.00 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a), other cond:le(test.t1.c, test.t2.b)", - "├─TableReader(Build) 1.00 root data:TableFullScan", - "│ └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false", - "└─IndexLookUp(Probe) 2.00 root ", - " ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(test.t2.a, test.t1.a) le(test.t1.c, test.t2.b)], keep order:false", - " └─TableRowIDScan(Probe) 2.00 cop[tikv] table:t2 keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a and t2.b = 1", - "Plan": [ - "IndexJoin 1.00 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", - "├─TableReader(Build) 1.00 root data:TableFullScan", - "│ └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false", - "└─IndexLookUp(Probe) 1.00 root ", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, index:PRIMARY(a, b) range: decided by [eq(test.t2.a, test.t1.a) eq(test.t2.b, 1)], keep order:false", - " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2 keep order:false" - ] - } - ] - }, { "Name": "TestPartitionTableStats", "Cases": [ @@ -332,44 +233,6 @@ } ] }, - { - "Name": "TestIndexMerge", - "Cases": [ - { - "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 1 or b = 2", - "Plan": [ - "IndexMerge 2.00 root type: union", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[2,2], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, A, B) */ * from t where a = 1 or b = 2", - "Plan": [ - "IndexMerge 2.00 root type: union", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false, stats:pseudo", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[2,2], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, primary) */ * from t where 1 or t.c", - "Plan": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ USE_INDEX_MERGE(t, a, b, c) */ * from t where 1 or t.a = 1 or t.b = 2", - "Plan": [ - "TableReader 10000.00 root data:Selection", - "└─Selection 10000.00 cop[tikv] or(1, or(eq(test.t.a, 1), eq(test.t.b, 2)))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" - ] - } - ] - }, { "Name": "TestSubqueryWithTopN", "Cases": [ @@ -421,59 +284,6 @@ } ] }, - { - "Name": "TestIndexJoinTableRange", - "Cases": [ - { - "SQL": "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.b", - "Plan": [ - "IndexJoin 12487.50 root inner join, inner:TableReader, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.b, test.t2.b)", - "├─IndexReader(Build) 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:idx_t1_b(b) keep order:false, stats:pseudo", - "└─TableReader(Probe) 9980.01 root data:Selection", - " └─Selection 9980.01 cop[tikv] not(isnull(test.t2.b))", - " └─TableRangeScan 9990.00 cop[tikv] table:t2 range: decided by [test.t1.a], keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.a and t1.b = t2.b", - "Plan": [ - "IndexJoin 12487.50 root inner join, inner:TableReader, outer key:test.t1.a, test.t1.b, inner key:test.t2.a, test.t2.a, equal cond:eq(test.t1.a, test.t2.a), eq(test.t1.b, test.t2.a), eq(test.t1.b, test.t2.b)", - "├─IndexReader(Build) 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:idx_t1_b(b) keep order:false, stats:pseudo", - "└─TableReader(Probe) 9980.01 root data:Selection", - " └─Selection 9980.01 cop[tikv] not(isnull(test.t2.b))", - " └─TableRangeScan 9990.00 cop[tikv] table:t2 range: decided by [test.t1.a test.t1.b], keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.a = t4.a where t4.b = 1", - "Plan": [ - "Projection 12.50 root test.t3.a, test.t3.b, test.t3.c, test.t4.a, test.t4.b, test.t4.c", - "└─IndexJoin 12.50 root inner join, inner:TableReader, outer key:test.t3.a, inner key:test.t4.a, equal cond:eq(test.t3.a, test.t4.a)", - " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9.99 root data:Selection", - " └─Selection 9.99 cop[tikv] eq(test.t4.b, 1)", - " └─TableRangeScan 9990.00 cop[tikv] table:t4 range: decided by [eq(test.t4.a, test.t3.a) eq(test.t4.b, 1)], keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ INL_JOIN(t4) */ * from t3 join t4 on t3.b = t4.b where t4.a = 1", - "Plan": [ - "Projection 12.50 root test.t3.a, test.t3.b, test.t3.c, test.t4.a, test.t4.b, test.t4.c", - "└─IndexJoin 12.50 root inner join, inner:TableReader, outer key:test.t3.b, inner key:test.t4.b, equal cond:eq(test.t3.b, test.t4.b)", - " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9.99 root data:Selection", - " └─Selection 9.99 cop[tikv] eq(test.t4.a, 1)", - " └─TableRangeScan 9990.00 cop[tikv] table:t4 range: decided by [eq(test.t4.b, test.t3.b) eq(test.t4.a, 1)], keep order:false, stats:pseudo" - ] - } - ] - }, { "Name": "TestPartitionPruningForInExpr", "Cases": [ @@ -593,240 +403,6 @@ } ] }, - { - "Name": "TestAccessPathOnClusterIndex", - "Cases": [ - { - "SQL": "select * from t1", - "Plan": [ - "TableReader 3.00 root data:TableFullScan", - "└─TableFullScan 3.00 cop[tikv] table:t1 keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11", - "2 222 2.2000000000 12", - "3 333 3.3000000000 13" - ] - }, - { - "SQL": "select * from t1 where t1.a >= 1 and t1.a < 4", - "Plan": [ - "TableReader 3.00 root data:TableRangeScan", - "└─TableRangeScan 3.00 cop[tikv] table:t1 range:[1,4), keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11", - "2 222 2.2000000000 12", - "3 333 3.3000000000 13" - ] - }, - { - "SQL": "select * from t1 where t1.a = 1 and t1.b < \"333\"", - "Plan": [ - "TableReader 0.82 root data:TableRangeScan", - "└─TableRangeScan 0.82 cop[tikv] table:t1 range:[1 -inf,1 \"333\"), keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11" - ] - }, - { - "SQL": "select t1.a, t1.b, t1.c from t1 where t1.c = 3.3", - "Plan": [ - "IndexReader 1.00 root index:IndexRangeScan", - "└─IndexRangeScan 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false" - ], - "Res": [ - "3 333 3.3000000000" - ] - }, - { - "SQL": "select t1.b, t1.c from t1 where t1.c = 2.2", - "Plan": [ - "IndexReader 1.00 root index:IndexRangeScan", - "└─IndexRangeScan 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false" - ], - "Res": [ - "222 2.2000000000" - ] - }, - { - "SQL": "select /*+ use_index(t1, c) */ * from t1", - "Plan": [ - "IndexLookUp 3.00 root ", - "├─IndexFullScan(Build) 3.00 cop[tikv] table:t1, index:c(c) keep order:false", - "└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t1 keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11", - "2 222 2.2000000000 12", - "3 333 3.3000000000 13" - ] - }, - { - "SQL": "select * from t1 use index(c) where t1.c in (2.2, 3.3)", - "Plan": [ - "IndexLookUp 2.00 root ", - "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], [3.3000000000,3.3000000000], keep order:false", - "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t1 keep order:false" - ], - "Res": [ - "2 222 2.2000000000 12", - "3 333 3.3000000000 13" - ] - }, - { - "SQL": "select * from t1 where t1.a = 1 order by b", - "Plan": [ - "TableReader 1.00 root data:TableRangeScan", - "└─TableRangeScan 1.00 cop[tikv] table:t1 range:[1,1], keep order:true" - ], - "Res": [ - "1 111 1.1000000000 11" - ] - }, - { - "SQL": "select * from t1 order by a, b limit 1", - "Plan": [ - "Limit 1.00 root offset:0, count:1", - "└─TableReader 1.00 root data:Limit", - " └─Limit 1.00 cop[tikv] offset:0, count:1", - " └─TableFullScan 1.00 cop[tikv] table:t1 keep order:true" - ], - "Res": [ - "1 111 1.1000000000 11" - ] - }, - { - "SQL": "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a >= 1 or t1.c = 2.2", - "Plan": [ - "IndexMerge 3.00 root type: union", - "├─TableRangeScan(Build) 3.00 cop[tikv] table:t1 range:[1,+inf], keep order:false", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false", - "└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t1 keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11", - "2 222 2.2000000000 12", - "3 333 3.3000000000 13" - ] - }, - { - "SQL": "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a = 1 and t1.b = '111' or t1.c = 3.3", - "Plan": [ - "IndexMerge 1.67 root type: union", - "├─TableRangeScan(Build) 1.00 cop[tikv] table:t1 range:[1 \"111\",1 \"111\"], keep order:false", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false", - "└─TableRowIDScan(Probe) 1.67 cop[tikv] table:t1 keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11", - "3 333 3.3000000000 13" - ] - } - ] - }, - { - "Name": "TestIndexJoinOnClusteredIndex", - "Cases": [ - { - "SQL": "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", - "Plan": [ - "IndexJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a, equal cond:eq(test.t.a, test.t.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", - "└─TableReader(Probe) 3.00 root data:TableRangeScan", - " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a)], keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11 1 111 1.1000000000 11", - "2 222 2.2000000000 12 2 222 2.2000000000 12", - "3 333 3.3000000000 13 3 333 3.3000000000 13" - ] - }, - { - "SQL": "select /*+ inl_merge_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", - "Plan": [ - "IndexMergeJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", - "└─TableReader(Probe) 3.00 root data:TableRangeScan", - " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a)], keep order:true" - ], - "Res": [ - "1 111 1.1000000000 11 1 111 1.1000000000 11", - "2 222 2.2000000000 12 2 222 2.2000000000 12", - "3 333 3.3000000000 13 3 333 3.3000000000 13" - ] - }, - { - "SQL": "select /*+ inl_hash_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a", - "Plan": [ - "IndexHashJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a, equal cond:eq(test.t.a, test.t.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", - "└─TableReader(Probe) 3.00 root data:TableRangeScan", - " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a)], keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11 1 111 1.1000000000 11", - "2 222 2.2000000000 12 2 222 2.2000000000 12", - "3 333 3.3000000000 13 3 333 3.3000000000 13" - ] - }, - { - "SQL": "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a and t1.b = t2.b", - "Plan": [ - "IndexJoin 3.00 root inner join, inner:TableReader, outer key:test.t.a, test.t.b, inner key:test.t.a, test.t.b, equal cond:eq(test.t.a, test.t.a), eq(test.t.b, test.t.b)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", - "└─TableReader(Probe) 3.00 root data:TableRangeScan", - " └─TableRangeScan 3.00 cop[tikv] table:t2 range: decided by [eq(test.t.a, test.t.a) eq(test.t.b, test.t.b)], keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11 1 111 1.1000000000 11", - "2 222 2.2000000000 12 2 222 2.2000000000 12", - "3 333 3.3000000000 13 3 333 3.3000000000 13" - ] - }, - { - "SQL": "select /*+ inl_join(t1, t2) */ * from t t1 join t t2 on t1.c = t2.c", - "Plan": [ - "IndexJoin 3.00 root inner join, inner:IndexLookUp, outer key:test.t.c, inner key:test.t.c, equal cond:eq(test.t.c, test.t.c)", - "├─TableReader(Build) 3.00 root data:Selection", - "│ └─Selection 3.00 cop[tikv] not(isnull(test.t.c))", - "│ └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false", - "└─IndexLookUp(Probe) 3.00 root ", - " ├─Selection(Build) 3.00 cop[tikv] not(isnull(test.t.c))", - " │ └─IndexRangeScan 3.00 cop[tikv] table:t2, index:c(c) range: decided by [eq(test.t.c, test.t.c)], keep order:false", - " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t2 keep order:false" - ], - "Res": [ - "1 111 1.1000000000 11 1 111 1.1000000000 11", - "2 222 2.2000000000 12 2 222 2.2000000000 12", - "3 333 3.3000000000 13 3 333 3.3000000000 13" - ] - }, - { - "SQL": "select /*+ inl_merge_join(t1,t2) */ t2.a, t2.c, t2.d from t t1 left join t t2 on t1.a = t2.c;", - "Plan": [ - "IndexMergeJoin 3.00 root left outer join, inner:Projection, outer key:Column#9, inner key:test.t.c", - "├─Projection(Build) 3.00 root cast(test.t.a, decimal(10,0) BINARY)->Column#9", - "│ └─IndexReader 3.00 root index:IndexFullScan", - "│ └─IndexFullScan 3.00 cop[tikv] table:t1, index:c(c) keep order:false", - "└─Projection(Probe) 3.00 root test.t.a, test.t.c, test.t.d", - " └─IndexLookUp 3.00 root ", - " ├─IndexRangeScan(Build) 3.00 cop[tikv] table:t2, index:c(c) range: decided by [eq(test.t.c, Column#9)], keep order:true", - " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t2 keep order:false" - ], - "Res": [ - " ", - " ", - " " - ] - } - ] - }, { "Name": "TestPartitionExplain", "Cases": [ @@ -1409,136 +985,6 @@ } ] }, - { - "Name": "TestHeuristicIndexSelection", - "Cases": [ - { - "SQL": "select * from t1 where a = 3 or a = 5", - "Plan": [ - "Batch_Point_Get_5 2.00 887.04 root table:t1 handle:[3 5], keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 handle of t1 is selected since the path only has point ranges" - ] - }, - { - "SQL": "select f, g from t1 where f = 2 and g in (3, 4, 5)", - "Plan": [ - "Batch_Point_Get_5 3.00 380.16 root table:t1, index:f_g(f, g) keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 unique index f_g of t1 is selected since the path only has point ranges with single scan" - ] - }, - { - "SQL": "select * from t1 where c = 1 and (d = 2 or d = 3) and e in (4, 5)", - "Plan": [ - "Batch_Point_Get_5 4.00 1774.08 root table:t1, index:c_d_e(c, d, e) keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 unique index c_d_e of t1 is selected since the path only has point ranges with double scan" - ] - }, - { - "SQL": "select f, g from t1 where f = 2 and g > 3", - "Plan": [ - "IndexReader_6 33.33 733.82 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 33.33 6783.33 cop[tikv] table:t1, index:f_g(f, g) range:(2 3,2 +inf], keep order:false, stats:pseudo" - ], - "Warnings": [ - "Note 1105 unique index f_g of t1 is selected since the path only fetches limited number of rows with single scan" - ] - }, - { - "SQL": "select a, b, c from t2 where a = 1 and b = 2 and c in (1, 2, 3, 4, 5)", - "Plan": [ - "Selection_6 0.01 289.88 root eq(test.t2.b, 2), in(test.t2.c, 1, 2, 3, 4, 5)", - "└─Point_Get_5 1.00 190.08 root table:t2, index:idx_a(a) " - ], - "Warnings": [ - "Note 1105 unique index idx_a of t2 is selected since the path only has point ranges with double scan" - ] - }, - { - "SQL": "select * from t3 where (a = 1 or a = 3) and b = 'xx'", - "Plan": [ - "Batch_Point_Get_5 2.00 1449.36 root table:t3, clustered index:PRIMARY(a, b) keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 handle of t3 is selected since the path only has point ranges" - ] - }, - { - "SQL": "select * from t4 where (a = 1 or a = 3) and b = 'xx'", - "Plan": [ - "Batch_Point_Get_5 2.00 1449.36 root table:t4, index:PRIMARY(a, b) keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 unique index PRIMARY of t4 is selected since the path only has point ranges with double scan" - ] - }, - { - "SQL": "select a, b from t3 where (a = 1 or a = 3) and b = 'xx'", - "Plan": [ - "Batch_Point_Get_5 2.00 1322.64 root table:t3, clustered index:PRIMARY(a, b) keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 handle of t3 is selected since the path only has point ranges" - ] - }, - { - "SQL": "select a, b from t4 where (a = 1 or a = 3) and b = 'xx'", - "Plan": [ - "Batch_Point_Get_5 2.00 1322.64 root table:t4, index:PRIMARY(a, b) keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 unique index PRIMARY of t4 is selected since the path only has point ranges with single scan" - ] - }, - { - "SQL": "update t1 set b = 2 where a = 4 or a = 6", - "Plan": [ - "Update_4 N/A N/A root N/A", - "└─Batch_Point_Get_6 2.00 887.04 root table:t1 handle:[4 6], keep order:false, desc:false" - ], - "Warnings": [ - "Note 1105 handle of t1 is selected since the path only has point ranges" - ] - }, - { - "SQL": "delete from t1 where f = 2 and g in (3, 4)", - "Plan": [ - "Delete_4 N/A N/A root N/A", - "└─Selection_7 2.00 493.42 root in(test.t1.g, 3, 4)", - " └─Point_Get_6 1.00 443.52 root table:t1, index:f(f) " - ], - "Warnings": [ - "Note 1105 unique index f of t1 is selected since the path only has point ranges with double scan" - ] - }, - { - "SQL": "insert into t3 select a, b, c from t1 where f = 2", - "Plan": [ - "Insert_1 N/A N/A root N/A", - "└─Projection_6 1.00 253.74 root test.t1.a, test.t1.b, test.t1.c", - " └─Point_Get_7 1.00 253.44 root table:t1, index:f(f) " - ], - "Warnings": [ - "Note 1105 unique index f of t1 is selected since the path only has point ranges with double scan" - ] - }, - { - "SQL": "replace into t3 select a, b, c from t1 where a = 3", - "Plan": [ - "Insert_1 N/A N/A root N/A", - "└─Point_Get_7 1.00 190.08 root table:t1 handle:3" - ], - "Warnings": [ - "Note 1105 handle of t1 is selected since the path only has point ranges" - ] - } - ] - }, { "Name": "TestOutputSkylinePruningInfo", "Cases": [ @@ -1821,172 +1267,55 @@ { "SQL": "select /*+ use_index_merge(tt1) */ 1 from tt1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c3)) = '10';", "Plan": [ - "Projection 15.99 root 1->Column#6", - "└─Selection 15.99 root or(eq(test.tt1.c1, \"de\"), and(eq(test.tt1.c2, \"10\"), eq(from_base64(to_base64(test.tt1.c3)), \"10\")))", - " └─IndexMerge 19.99 root type: union", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:tt1, index:idx_0(c1) range:[\"de\",\"de\"], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:tt1, index:idx_1(c2, c3) range:[\"10\",\"10\"], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 19.99 cop[tikv] table:tt1 keep order:false, stats:pseudo" - ], - "Res": [ - "1" - ] - }, - { - "SQL": "select /*+ use_index_merge( tt2 ) */ 1 from tt2 where tt2.c1 in (-3896405) or tt2.pk in (1, 53330) and to_base64(left(pk, 5));", - "Plan": [ - "Projection 2.40 root 1->Column#3", - "└─Selection 2.40 root or(eq(test.tt2.c1, -3896405), and(in(test.tt2.pk, 1, 53330), istrue_with_null(cast(to_base64(left(cast(test.tt2.pk, var_string(20)), 5)), double BINARY))))", - " └─IndexMerge 3.00 root type: union", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:tt2, index:c1(c1) range:[-3896405,-3896405], keep order:false, stats:pseudo", - " ├─TableRangeScan(Build) 2.00 cop[tikv] table:tt2 range:[1,1], [53330,53330], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:tt2 keep order:false, stats:pseudo" - ], - "Res": [ - "1" - ] - }, - { - "SQL": "select /*+ use_index_merge(tt3) */ 1 from tt3 where c1 < -10 or c2 < 10 and reverse(c3) = '2';", - "Plan": [ - "Projection 5098.44 root 1->Column#5", - "└─Selection 2825.66 root or(lt(test.tt3.c1, -10), and(lt(test.tt3.c2, 10), eq(reverse(cast(test.tt3.c3, var_string(20))), \"2\")))", - " └─IndexMerge 5542.21 root type: union", - " ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:tt3, index:c1(c1) range:[-inf,-10), keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:tt3, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 5542.21 cop[tikv] table:tt3 keep order:false, stats:pseudo" - ], - "Res": [ - "1" - ] - }, - { - "SQL": "select 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab';", - "Plan": [ - "Projection 8000.00 root 1->Column#5", - "└─Selection 8000.00 root or(eq(test.t1.c1, \"de\"), and(eq(test.t1.c2, \"10\"), eq(from_base64(to_base64(test.t1.c1)), \"ab\")))", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Res": [ - "1" - ] - } - ] - }, - { - "Name": "TestIndexMergeWithCorrelatedColumns", - "Cases": [ - { - "SQL": "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", - "Plan": [ - "Sort 10000.00 root test.t2.c1", - "└─Projection 10000.00 root test.t2.c1, test.t2.c2, test.t2.c3", - " └─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.t2.c1, Column#8), if(ne(Column#9, 0), NULL, 1)), or(eq(Column#10, 0), if(isnull(test.t2.c1), NULL, 0)))", - " ├─TableReader(Build) 10000.00 root data:TableFullScan", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", - " └─IndexMerge 63.35 root type: union", - " ├─Selection(Build) 10000.00 cop[tikv] eq(10, test.t2.c3)", - " │ └─TableRangeScan 10000.00 cop[tikv] table:t1 range:[10,10], keep order:false, stats:pseudo", - " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", - " │ └─IndexRangeScan 100000.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 63.35 cop[tikv] or(and(eq(test.t1.c1, 10), eq(10, test.t2.c3)), and(eq(test.t1.c2, 1), eq(1, test.t2.c3))), substring(cast(test.t1.c3, var_string(20)), 10)", - " └─TableRowIDScan 89992.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Res": [ - "1 1 1", - "2 2 2" - ] - }, - { - "SQL": "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 = 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and reverse(c3)) order by c1;", - "Plan": [ - "Sort 10000.00 root test.t2.c1", - "└─Projection 10000.00 root test.t2.c1, test.t2.c2, test.t2.c3", - " └─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.t2.c1, Column#8), if(ne(Column#9, 0), NULL, 1)), or(eq(Column#10, 0), if(isnull(test.t2.c1), NULL, 0)))", - " ├─TableReader(Build) 10000.00 root data:TableFullScan", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", - " └─IndexMerge 63.35 root type: union", - " ├─Selection(Build) 10000.00 cop[tikv] eq(10, test.t2.c3)", - " │ └─TableRangeScan 10000.00 cop[tikv] table:t1 range:[10,10], keep order:false, stats:pseudo", - " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", - " │ └─IndexRangeScan 100000.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 63.35 cop[tikv] or(and(eq(test.t1.c1, 10), eq(10, test.t2.c3)), and(eq(test.t1.c2, 1), eq(1, test.t2.c3))), reverse(cast(test.t1.c3, var_string(20)))", - " └─TableRowIDScan 89992.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + "Projection 15.99 root 1->Column#6", + "└─Selection 15.99 root or(eq(test.tt1.c1, \"de\"), and(eq(test.tt1.c2, \"10\"), eq(from_base64(to_base64(test.tt1.c3)), \"10\")))", + " └─IndexMerge 19.99 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:tt1, index:idx_0(c1) range:[\"de\",\"de\"], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:tt1, index:idx_1(c2, c3) range:[\"10\",\"10\"], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 19.99 cop[tikv] table:tt1 keep order:false, stats:pseudo" ], "Res": [ - "2 2 2" + "1" ] }, { - "SQL": "select * from t2 where c1 < all(select /*+ use_index_merge(t1) */ c1 from t1 where (c1 >= 10 and c1 = t2.c3 or c2 = 1 and c2 = t2.c3) and substring(c3, 10)) order by c1;", + "SQL": "select /*+ use_index_merge( tt2 ) */ 1 from tt2 where tt2.c1 in (-3896405) or tt2.pk in (1, 53330) and to_base64(left(pk, 5));", "Plan": [ - "Sort 10000.00 root test.t2.c1", - "└─Projection 10000.00 root test.t2.c1, test.t2.c2, test.t2.c3", - " └─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.t2.c1, Column#8), if(ne(Column#9, 0), NULL, 1)), or(eq(Column#10, 0), if(isnull(test.t2.c1), NULL, 0)))", - " ├─TableReader(Build) 10000.00 root data:TableFullScan", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─StreamAgg(Probe) 10000.00 root funcs:min(test.t1.c1)->Column#8, funcs:sum(0)->Column#9, funcs:count(1)->Column#10", - " └─IndexMerge 30263.46 root type: union", - " ├─Selection(Build) 33333.33 cop[tikv] eq(test.t1.c1, test.t2.c3)", - " │ └─TableRangeScan 33333333.33 cop[tikv] table:t1 range:[10,+inf], keep order:false, stats:pseudo", - " ├─Selection(Build) 80000.00 cop[tikv] eq(1, test.t2.c3)", - " │ └─IndexRangeScan 100000.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:pseudo", - " └─Selection(Probe) 30263.46 cop[tikv] or(and(ge(test.t1.c1, 10), eq(test.t1.c1, test.t2.c3)), and(eq(test.t1.c2, 1), eq(1, test.t2.c3))), substring(cast(test.t1.c3, var_string(20)), 10)", - " └─TableRowIDScan 33386666.67 cop[tikv] table:t1 keep order:false, stats:pseudo" + "Projection 2.40 root 1->Column#3", + "└─Selection 2.40 root or(eq(test.tt2.c1, -3896405), and(in(test.tt2.pk, 1, 53330), istrue_with_null(cast(to_base64(left(cast(test.tt2.pk, var_string(20)), 5)), double BINARY))))", + " └─IndexMerge 3.00 root type: union", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:tt2, index:c1(c1) range:[-3896405,-3896405], keep order:false, stats:pseudo", + " ├─TableRangeScan(Build) 2.00 cop[tikv] table:tt2 range:[1,1], [53330,53330], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:tt2 keep order:false, stats:pseudo" ], "Res": [ - "1 1 1", - "2 2 2" + "1" ] }, { - "SQL": "select c_int from tt1 where c_decimal < all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt1.c_int = tt2.c_int and tt1.c_datetime > tt2.c_datetime and tt2.c_decimal = 9.060 or tt2.c_str <= 'interesting shtern' and tt1.c_int = tt2.c_int) order by 1;", + "SQL": "select /*+ use_index_merge(tt3) */ 1 from tt3 where c1 < -10 or c2 < 10 and reverse(c3) = '2';", "Plan": [ - "Projection 10000.00 root test.tt1.c_int", - "└─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(lt(test.tt1.c_decimal, Column#9), if(ne(Column#10, 0), NULL, 1)), or(eq(Column#11, 0), if(isnull(test.tt1.c_decimal), NULL, 0)))", - " ├─TableReader(Build) 10000.00 root data:TableFullScan", - " │ └─TableFullScan 10000.00 cop[tikv] table:tt1 keep order:true, stats:pseudo", - " └─StreamAgg(Probe) 10000.00 root funcs:min(Column#14)->Column#9, funcs:sum(Column#15)->Column#10, funcs:count(1)->Column#11", - " └─Projection 11.05 root test.tt2.c_decimal->Column#14, cast(isnull(test.tt2.c_decimal), decimal(20,0) BINARY)->Column#15", - " └─IndexMerge 11.05 root type: union", - " ├─Selection(Build) 10.00 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", - " │ └─IndexRangeScan 10000.00 cop[tikv] table:tt2, index:c_decimal(c_decimal) range:[9.060000,9.060000], keep order:false, stats:pseudo", - " ├─Selection(Build) 33233.33 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", - " │ └─IndexRangeScan 33233333.33 cop[tikv] table:tt2, index:c_str(c_str) range:[-inf,\"interesting shtern\"], keep order:false, stats:pseudo", - " └─Selection(Probe) 11.05 cop[tikv] or(and(eq(test.tt1.c_int, test.tt2.c_int), and(gt(test.tt1.c_datetime, test.tt2.c_datetime), eq(test.tt2.c_decimal, 9.060))), and(le(test.tt2.c_str, \"interesting shtern\"), eq(test.tt1.c_int, test.tt2.c_int)))", - " └─TableRowIDScan 33243.33 cop[tikv] table:tt2 keep order:false, stats:pseudo" + "Projection 5098.44 root 1->Column#5", + "└─Selection 2825.66 root or(lt(test.tt3.c1, -10), and(lt(test.tt3.c2, 10), eq(reverse(cast(test.tt3.c3, var_string(20))), \"2\")))", + " └─IndexMerge 5542.21 root type: union", + " ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:tt3, index:c1(c1) range:[-inf,-10), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:tt3, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 5542.21 cop[tikv] table:tt3 keep order:false, stats:pseudo" ], "Res": [ - "7", - "8", - "10" + "1" ] }, { - "SQL": "select c_int from tt1 where c_decimal > all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt2.c_int = 7 and tt2.c_int < tt1.c_decimal or tt2.c_str >= 'zzzzzzzzzzzzzzzzzzz' and tt1.c_int = tt2.c_int) order by 1;", + "SQL": "select 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab';", "Plan": [ - "Projection 10000.00 root test.tt1.c_int", - "└─Apply 10000.00 root CARTESIAN inner join, other cond:or(and(gt(test.tt1.c_decimal, Column#9), if(ne(Column#10, 0), NULL, 1)), or(eq(Column#11, 0), if(isnull(test.tt1.c_decimal), NULL, 0)))", - " ├─TableReader(Build) 10000.00 root data:TableFullScan", - " │ └─TableFullScan 10000.00 cop[tikv] table:tt1 keep order:true, stats:pseudo", - " └─StreamAgg(Probe) 10000.00 root funcs:max(Column#14)->Column#9, funcs:sum(Column#15)->Column#10, funcs:count(1)->Column#11", - " └─Projection 17.91 root test.tt2.c_decimal->Column#14, cast(isnull(test.tt2.c_decimal), decimal(20,0) BINARY)->Column#15", - " └─IndexMerge 17.91 root type: union", - " ├─Selection(Build) 10000.00 cop[tikv] lt(7, test.tt1.c_decimal)", - " │ └─TableRangeScan 10000.00 cop[tikv] table:tt2 range:[7,7], keep order:false, stats:pseudo", - " ├─Selection(Build) 33333.33 cop[tikv] eq(test.tt1.c_int, test.tt2.c_int)", - " │ └─IndexRangeScan 33333333.33 cop[tikv] table:tt2, index:c_str(c_str) range:[\"zzzzzzzzzzzzzzzzzzz\",+inf], keep order:false, stats:pseudo", - " └─Selection(Probe) 17.91 cop[tikv] or(and(eq(test.tt2.c_int, 7), lt(7, test.tt1.c_decimal)), and(ge(test.tt2.c_str, \"zzzzzzzzzzzzzzzzzzz\"), eq(test.tt1.c_int, test.tt2.c_int)))", - " └─TableRowIDScan 43330.00 cop[tikv] table:tt2 keep order:false, stats:pseudo" + "Projection 8000.00 root 1->Column#5", + "└─Selection 8000.00 root or(eq(test.t1.c1, \"de\"), and(eq(test.t1.c2, \"10\"), eq(from_base64(to_base64(test.t1.c1)), \"ab\")))", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Res": [ - "6", - "7", - "8", - "9" + "1" ] } ] @@ -2567,83 +1896,6 @@ } ] }, - { - "Name": "TestIndexMergeSerial", - "Cases": [ - { - "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", - "Plan": [ - "IndexMerge 8.00 root type: union", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - "├─Selection(Build) 1.00 cop[tikv] 1", - "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" - ], - "Warnings": null - }, - { - "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", - "Plan": [ - "IndexMerge 8.00 root type: union", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - "├─Selection(Build) 1.00 cop[tikv] 1", - "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" - ], - "Warnings": null - }, - { - "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", - "Plan": [ - "IndexMerge 8.00 root type: union", - "├─Selection(Build) 1.00 cop[tikv] 1", - "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - "├─Selection(Build) 1.00 cop[tikv] 1", - "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" - ], - "Warnings": null - }, - { - "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)", - "Plan": [ - "IndexMerge 0.29 root type: union", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - "└─Selection(Probe) 0.29 cop[tikv] or(and(eq(test.t.a, 1), eq(length(cast(test.t.b, var_string(20))), 1)), and(eq(test.t.b, 1), eq(length(cast(test.t.a, var_string(20))), 1)))", - " └─TableRowIDScan 1.90 cop[tikv] table:t keep order:false" - ], - "Warnings": null - } - ] - }, - { - "Name": "TestLimitIndexLookUpKeepOrder", - "Cases": [ - { - "SQL": "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", - "Plan": [ - "Limit 0.00 root offset:0, count:10", - "└─Projection 0.00 root test.t.a, test.t.b, test.t.c, test.t.d", - " └─IndexLookUp 0.00 root ", - " ├─IndexRangeScan(Build) 2.50 cop[tikv] table:t, index:idx(a, b, c) range:(1 2,1 10), keep order:true, stats:pseudo", - " └─Selection(Probe) 0.00 cop[tikv] eq(test.t.d, 10)", - " └─TableRowIDScan 2.50 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10", - "Plan": [ - "Limit 0.00 root offset:0, count:10", - "└─Projection 0.00 root test.t.a, test.t.b, test.t.c, test.t.d", - " └─IndexLookUp 0.00 root ", - " ├─IndexRangeScan(Build) 2.50 cop[tikv] table:t, index:idx(a, b, c) range:(1 2,1 10), keep order:true, desc, stats:pseudo", - " └─Selection(Probe) 0.00 cop[tikv] eq(test.t.d, 10)", - " └─TableRowIDScan 2.50 cop[tikv] table:t keep order:false, stats:pseudo" - ] - } - ] - }, { "Name": "TestIssue23887", "Cases": [ @@ -3649,380 +2901,6 @@ } ] }, - { - "Name": "TestIndexJoinRangeFallback", - "Cases": [ - { - "SQL": "set @@tidb_opt_range_max_size = 0", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "Plan": [ - "IndexJoin 0.50 root inner join, inner:IndexLookUp, outer key:test.t2.e, test.t2.g, inner key:test.t1.b, test.t1.d, equal cond:eq(test.t2.e, test.t1.b), eq(test.t2.g, test.t1.d)", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 0.50 root ", - " ├─Selection(Build) 249.50 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", - " │ └─IndexRangeScan 250.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.b, test.t2.e) eq(test.t1.d, test.t2.g) in(test.t1.a, 1, 3) in(test.t1.c, aaa, bbb)], keep order:false, stats:pseudo", - " └─Selection(Probe) 0.50 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", - " └─TableRowIDScan 249.50 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "set @@tidb_opt_range_max_size = 2900", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "Plan": [ - "IndexJoin 0.50 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.b, equal cond:eq(test.t2.e, test.t1.b), eq(test.t2.g, test.t1.d)", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 0.50 root ", - " ├─Selection(Build) 249.50 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", - " │ └─IndexRangeScan 250.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.b, test.t2.e) in(test.t1.a, 1, 3) in(test.t1.c, aaa, bbb)], keep order:false, stats:pseudo", - " └─Selection(Probe) 0.50 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", - " └─TableRowIDScan 249.50 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": [ - "Memory capacity of 2900 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" - ] - }, - { - "SQL": "set @@tidb_opt_range_max_size = 2300", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "Plan": [ - "IndexJoin 0.50 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.b, equal cond:eq(test.t2.e, test.t1.b), eq(test.t2.g, test.t1.d)", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 0.50 root ", - " ├─Selection(Build) 249.50 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", - " │ └─IndexRangeScan 250.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.b, test.t2.e) in(test.t1.a, 1, 3)], keep order:false, stats:pseudo", - " └─Selection(Probe) 0.50 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", - " └─TableRowIDScan 249.50 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": [ - "Memory capacity of 2300 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" - ] - }, - { - "SQL": "set @@tidb_opt_range_max_size = 700", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.b = t2.e and t1.d = t2.g where t1.a in (1, 3) and t1.c in ('aaa', 'bbb')", - "Plan": [ - "HashJoin 0.05 root inner join, equal:[eq(test.t1.b, test.t2.e) eq(test.t1.d, test.t2.g)]", - "├─IndexLookUp(Build) 0.04 root ", - "│ ├─Selection(Build) 19.96 cop[tikv] not(isnull(test.t1.b)), not(isnull(test.t1.d))", - "│ │ └─IndexRangeScan 20.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range:[1,1], [3,3], keep order:false, stats:pseudo", - "│ └─Selection(Probe) 0.04 cop[tikv] in(test.t1.c, \"aaa\", \"bbb\")", - "│ └─TableRowIDScan 19.96 cop[tikv] table:t1 keep order:false, stats:pseudo", - "└─TableReader(Probe) 9980.01 root data:Selection", - " └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.g))", - " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" - ], - "Warn": [ - "Memory capacity of 700 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", - "[planner:1815]Optimizer Hint /*+ INL_JOIN(t1) */ or /*+ TIDB_INLJ(t1) */ is inapplicable" - ] - }, - { - "SQL": "set @@tidb_opt_range_max_size = 0", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", - "Plan": [ - "IndexJoin 312.19 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a)", - "├─TableReader(Build) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.e))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 312.19 root ", - " ├─Selection(Build) 312.19 cop[tikv] not(isnull(test.t1.a))", - " │ └─IndexRangeScan 312.50 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e) gt(test.t1.b, 1) lt(test.t1.b, 10)], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 312.19 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "set @@tidb_opt_range_max_size = 300", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > 1 and t1.b < 10", - "Plan": [ - "IndexJoin 312.19 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a)", - "├─TableReader(Build) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.e))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 312.19 root ", - " ├─Selection(Build) 312.19 cop[tikv] gt(test.t1.b, 1), lt(test.t1.b, 10), not(isnull(test.t1.a))", - " │ └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e)], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 312.19 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": [ - "Memory capacity of 300 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" - ] - }, - { - "SQL": "set @@tidb_opt_range_max_size = 0", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10", - "Plan": [ - "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a), other cond:gt(test.t1.b, test.t2.f), lt(test.t1.b, plus(test.t2.f, 10))", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.f))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 12475.01 root ", - " ├─Selection(Build) 12475.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " │ └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e) lt(test.t1.b, plus(test.t2.f, 10)) gt(test.t1.b, test.t2.f)], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "set @@tidb_opt_range_max_size = 300", - "Plan": null, - "Warn": null - }, - { - "SQL": "explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.e where t1.b > t2.f and t1.b < t2.f + 10", - "Plan": [ - "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t2.e, inner key:test.t1.a, equal cond:eq(test.t2.e, test.t1.a), other cond:gt(test.t1.b, test.t2.f), lt(test.t1.b, plus(test.t2.f, 10))", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.e)), not(isnull(test.t2.f))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 12475.01 root ", - " ├─Selection(Build) 12475.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " │ └─IndexRangeScan 12500.00 cop[tikv] table:t1, index:idx_a_b_c_d(a, b, c, d) range: decided by [eq(test.t1.a, test.t2.e)], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 12475.01 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": [ - "Memory capacity of 300 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen" - ] - } - ] - }, - { - "Name": "TestNullConditionForPrefixIndex", - "Cases": [ - { - "SQL": "select count(1) from t1 where c1 = '0xfff' and c2 is not null", - "Plan": [ - "StreamAgg 1.00 root funcs:count(Column#7)->Column#5", - "└─IndexReader 1.00 root index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#7", - " └─IndexRangeScan 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo" - ], - "Result": [ - "3" - ] - }, - { - "SQL": "select count(1) from t1 where c1 = '0xfff' and c2 is null", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#5", - "└─IndexReader 0.10 root index:IndexRangeScan", - " └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" NULL,\"0xfff\" NULL], keep order:false, stats:pseudo" - ], - "Result": [ - "1" - ] - }, - { - "SQL": "select count(1) from t1 where c1 >= '0xfff' and c2 is not null", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#6)->Column#5", - "└─IndexReader 1.00 root index:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#6", - " └─Selection 3330.00 cop[tikv] not(isnull(test.t1.c2))", - " └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo" - ], - "Result": [ - "3" - ] - }, - { - "SQL": "select count(1) from t1 where c1 >= '0xfff' and c2 is null", - "Plan": [ - "StreamAgg 1.00 root funcs:count(Column#7)->Column#5", - "└─IndexReader 1.00 root index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#7", - " └─Selection 3.33 cop[tikv] isnull(test.t1.c2)", - " └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo" - ], - "Result": [ - "1" - ] - }, - { - "SQL": "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is not null", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#5", - "└─IndexLookUp 8.00 root ", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:idx1(c1) range:[\"0xfff\",\"0xfff\"], keep order:false, stats:pseudo", - " └─Selection(Probe) 8.00 cop[tikv] not(isnull(plus(cast(test.t1.c2, double BINARY), 1)))", - " └─TableRowIDScan 10.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Result": [ - "3" - ] - }, - { - "SQL": "select count(1) from t1 where c1 = '0xfff' and (c2 + 1) is null", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#5", - "└─IndexLookUp 8.00 root ", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:idx1(c1) range:[\"0xfff\",\"0xfff\"], keep order:false, stats:pseudo", - " └─Selection(Probe) 8.00 cop[tikv] isnull(plus(cast(test.t1.c2, double BINARY), 1))", - " └─TableRowIDScan 10.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Result": [ - "1" - ] - }, - { - "SQL": "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is not null", - "Plan": [ - "Projection 99.90 root test.t1.c2", - "└─IndexLookUp 99.90 root ", - " ├─IndexRangeScan(Build) 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 99.90 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Result": [ - "", - "111111", - "22 " - ] - }, - { - "SQL": "select c2 from t1 use index(idx2) where c1 = '0xfff' and c2 is null", - "Plan": [ - "Projection 0.10 root test.t1.c2", - "└─IndexLookUp 0.10 root ", - " ├─IndexRangeScan(Build) 0.10 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" NULL,\"0xfff\" NULL], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 0.10 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Result": [ - "" - ] - }, - { - "SQL": "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is not null", - "Plan": [ - "Projection 3330.00 root test.t1.c2", - "└─IndexLookUp 3330.00 root ", - " ├─Selection(Build) 3330.00 cop[tikv] not(isnull(test.t1.c2))", - " │ └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 3330.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Result": [ - "", - "111111", - "22 " - ] - }, - { - "SQL": "select c2 from t1 use index(idx2) where c1 >= '0xfff' and c2 is null", - "Plan": [ - "Projection 3.33 root test.t1.c2", - "└─IndexLookUp 3.33 root ", - " ├─Selection(Build) 3.33 cop[tikv] isnull(test.t1.c2)", - " │ └─IndexRangeScan 3333.33 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\",+inf], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 3.33 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Result": [ - "" - ] - }, - { - "SQL": "select count(1) from t2 use index(idx) where b is not null", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#5)->Column#4", - "└─IndexReader 1.00 root index:HashAgg", - " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#5", - " └─IndexFullScan 9990.00 cop[tikv] table:t2, index:idx(b) keep order:false, stats:pseudo" - ], - "Result": [ - "3" - ] - }, - { - "SQL": "select count(1) from t2 use index(idx) where b is null", - "Plan": [ - "StreamAgg 1.00 root funcs:count(Column#6)->Column#4", - "└─IndexReader 1.00 root index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#6", - " └─IndexRangeScan 10.00 cop[tikv] table:t2, index:idx(b) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - "1" - ] - }, - { - "SQL": "select b from t2 use index(idx) where b is not null", - "Plan": [ - "IndexLookUp 9990.00 root ", - "├─IndexFullScan(Build) 9990.00 cop[tikv] table:t2, index:idx(b) keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 9990.00 cop[tikv] table:t2 keep order:false, stats:pseudo" - ], - "Result": [ - "", - "aaaaaa", - "bb " - ] - }, - { - "SQL": "select b from t2 use index(idx) where b is null", - "Plan": [ - "IndexLookUp 10.00 root ", - "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, index:idx(b) range:[NULL,NULL], keep order:false, stats:pseudo", - "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t2 keep order:false, stats:pseudo" - ], - "Result": [ - "" - ] - }, - { - "SQL": "select b from t3 where a = 1 and b is not null", - "Plan": [ - "Projection 10.00 root test.t3.b", - "└─TableReader 10.00 root data:TableRangeScan", - " └─TableRangeScan 10.00 cop[tikv] table:t3 range:[1,1], keep order:false, stats:pseudo" - ], - "Result": [ - "", - "aaaaaa", - "bb " - ] - }, - { - "SQL": "select b from t3 where a = 1 and b is null", - "Plan": [ - "TableDual 0.00 root rows:0" - ], - "Result": null - } - ] - }, { "Name": "TestDowncastPointGetOrRangeScan", "Cases": [ From 0eb45605adda047f2c1f1799737e1d8797a3188f Mon Sep 17 00:00:00 2001 From: Pierre-Alexandre VEYRY Date: Fri, 8 Sep 2023 16:27:07 +0100 Subject: [PATCH 8/9] util: memory: fix segfaults in meminfo when VirtualMemory errors (#46762) close pingcap/tidb#46761 --- util/memory/meminfo.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/util/memory/meminfo.go b/util/memory/meminfo.go index 50dcf555db77e..e1dccb2530fbc 100644 --- a/util/memory/meminfo.go +++ b/util/memory/meminfo.go @@ -53,7 +53,7 @@ func MemTotalNormal() (uint64, error) { } v, err := mem.VirtualMemory() if err != nil { - return v.Total, err + return 0, err } memLimit.set(v.Total, time.Now()) return v.Total, nil @@ -67,7 +67,7 @@ func MemUsedNormal() (uint64, error) { } v, err := mem.VirtualMemory() if err != nil { - return v.Used, err + return 0, err } memUsage.set(v.Used, time.Now()) return v.Used, nil From d3d30f5f556e826ae5ec2d25c8d8fd6b9412e72a Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Sat, 9 Sep 2023 00:18:06 +0800 Subject: [PATCH 9/9] session: add debug log for parsing (#46638) close pingcap/tidb#46637 --- session/session.go | 1 + 1 file changed, 1 insertion(+) diff --git a/session/session.go b/session/session.go index 8a3fcf5159031..278d5519804d3 100644 --- a/session/session.go +++ b/session/session.go @@ -1678,6 +1678,7 @@ func (s *session) Execute(ctx context.Context, sql string) (recordSets []sqlexec // Parse parses a query string to raw ast.StmtNode. func (s *session) Parse(ctx context.Context, sql string) ([]ast.StmtNode, error) { + logutil.Logger(ctx).Debug("parse", zap.String("sql", sql)) parseStartTime := time.Now() stmts, warns, err := s.ParseSQL(ctx, sql, s.sessionVars.GetParseParams()...) if err != nil {