From f6b953c340c8087b4a70371b87407d36078315ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20van=20Eeden?= Date: Thu, 17 Aug 2023 15:24:31 +0200 Subject: [PATCH 01/32] expression: WEIGHT_STRING() collation (#45733) close pingcap/tidb#45725 --- expression/builtin_string.go | 1 + expression/builtin_string_test.go | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/expression/builtin_string.go b/expression/builtin_string.go index eea726d5989e9..0c456b4272fcb 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -3959,6 +3959,7 @@ func (c *weightStringFunctionClass) getFunction(ctx sessionctx.Context, args []E if err != nil { return nil, err } + types.SetBinChsClnFlag(bf.tp) var sig builtinFunc if padding == weightStringPaddingNull { sig = &builtinWeightStringNullSig{bf} diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index eaec95a3c2705..9356ac36f57ac 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -2604,6 +2604,10 @@ func TestWeightString(t *testing.T) { f, err = fc.getFunction(ctx, datumsToConstants([]types.Datum{str, padding, length})) } require.NoError(t, err) + + retType := f.getRetTp() + require.Equal(t, charset.CollationBin, retType.GetCollate()) + // Reset warnings. ctx.GetSessionVars().StmtCtx.ResetForRetry() result, err := evalBuiltinFunc(f, chunk.Row{}) From 8298ae51b74ca8845b82b7842cbbeed6a8da493b Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 17 Aug 2023 22:53:01 +0800 Subject: [PATCH 02/32] linter: enable toomanytests for br/pkg (#46186) ref pingcap/tidb#40786 --- build/nogo_config.json | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/build/nogo_config.json b/build/nogo_config.json index 99dad8e2114b9..97c68b6f486a5 100644 --- a/build/nogo_config.json +++ b/build/nogo_config.json @@ -602,6 +602,27 @@ "ddl/tests/partition/": "more than 50" }, "only_files": { + "br/pkg/version/": "br/pkg/version/ coded", + "br/pkg/utils/storewatch/": "br/pkg/utils/storewatch/ coded", + "br/pkg/utils/iter/": "br/pkg/utils/iter/ coded", + "br/pkg/utils/": "br/pkg/utils/ coded", + "br/pkg/task/show/": "br/pkg/task/show/ coded", + "br/pkg/task/": "br/pkg/task/ coded", + "br/pkg/streamhelper/spans/": "br/pkg/streamhelper/spans/ coded", + "br/pkg/streamhelper/": "br/pkg/streamhelper/ coded", + "br/pkg/stream/": "br/pkg/stream/ coded", + "br/pkg/storage/": "br/pkg/storage/ coded", + "br/pkg/rtree/": "br/pkg/rtree/ coded", + "br/pkg/restore/tiflashrec/": "br/pkg/restore/tiflashrec/ coded", + "br/pkg/restore/split/": "br/pkg/restore/split/ coded", + "br/pkg/restore/ingestrec/": "br/pkg/restore/ingestrec/ coded", + "br/pkg/pdutil/": "br/pkg/pdutil/ coded", + "br/pkg/metautil/": "br/pkg/metautil/ coded", + "br/pkg/logutil/": "br/pkg/logutil/ coded", + "br/pkg/gluetidb/": "br/pkg/gluetidb/ coded", + "br/pkg/conn/": "br/pkg/conn/ coded", + "br/pkg/checkpoint/": "br/pkg/checkpoint/ coded", + "br/pkg/backup/": "br/pkg/backup/ coded", "planer/core/casetest/binaryplan": "planer/core/casetest/binaryplan", "planer/core/casetest/cbotest": "planer/core/casetest/cbotest", "planer/core/casetest/enforcempp": "planer/core/casetest/enforcempp", From 9f23bae3f0f645f4107fea284736735d21d2a73b Mon Sep 17 00:00:00 2001 From: Changjie Liu Date: Fri, 18 Aug 2023 00:39:31 +0800 Subject: [PATCH 03/32] ddl: fix show global index regions (#45999) close pingcap/tidb#46000 --- ddl/tests/partition/db_partition_test.go | 31 ++++++++++++++++++++++++ executor/show.go | 17 ++++++++++++- 2 files changed, 47 insertions(+), 1 deletion(-) diff --git a/ddl/tests/partition/db_partition_test.go b/ddl/tests/partition/db_partition_test.go index abf22c951aaa1..aa7787222466f 100644 --- a/ddl/tests/partition/db_partition_test.go +++ b/ddl/tests/partition/db_partition_test.go @@ -2762,6 +2762,37 @@ func TestGlobalIndexLookUpInDropPartition(t *testing.T) { indexLookupResult.Check(testkit.Rows("11 11 11", "12 12 12")) } +func TestGlobalIndexShowTableRegions(t *testing.T) { + atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1) + defer atomic.StoreUint32(&ddl.EnableSplitTableRegion, 0) + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.EnableGlobalIndex = true + }) + + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists p") + tk.MustExec("set @@global.tidb_scatter_region = on") + tk.MustExec(`create table p (id int, c int, d int, unique key uidx(c)) partition by range (c) ( +partition p0 values less than (4), +partition p1 values less than (7), +partition p2 values less than (10))`) + rs := tk.MustQuery("show table p regions").Rows() + require.Equal(t, len(rs), 3) + rs = tk.MustQuery("show table p index uidx regions").Rows() + require.Equal(t, len(rs), 3) + + tk.MustExec("alter table p add unique idx(id)") + rs = tk.MustQuery("show table p regions").Rows() + require.Equal(t, len(rs), 4) + rs = tk.MustQuery("show table p index idx regions").Rows() + require.Equal(t, len(rs), 1) + rs = tk.MustQuery("show table p index uidx regions").Rows() + require.Equal(t, len(rs), 3) +} + func TestAlterTableExchangePartition(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/executor/show.go b/executor/show.go index 53bd8cddccbbe..5ee5b15ae5ac1 100644 --- a/executor/show.go +++ b/executor/show.go @@ -2011,6 +2011,7 @@ func (e *ShowExec) fetchShowTableRegions(ctx context.Context) error { } physicalIDs := []int64{} + hasGlobalIndex := false if pi := tb.Meta().GetPartitionInfo(); pi != nil { for _, name := range e.Table.PartitionNames { pid, err := tables.FindPartitionByName(tb.Meta(), name.L) @@ -2024,6 +2025,13 @@ func (e *ShowExec) fetchShowTableRegions(ctx context.Context) error { physicalIDs = append(physicalIDs, p.ID) } } + // when table has global index, show the logical table region. + for _, index := range tb.Meta().Indices { + if index.Global { + hasGlobalIndex = true + break + } + } } else { if len(e.Table.PartitionNames) != 0 { return plannercore.ErrPartitionClauseOnNonpartitioned @@ -2039,9 +2047,16 @@ func (e *ShowExec) fetchShowTableRegions(ctx context.Context) error { if indexInfo == nil { return plannercore.ErrKeyDoesNotExist.GenWithStackByArgs(e.IndexName, tb.Meta().Name) } - regions, err = getTableIndexRegions(indexInfo, physicalIDs, tikvStore, splitStore) + if indexInfo.Global { + regions, err = getTableIndexRegions(indexInfo, []int64{tb.Meta().ID}, tikvStore, splitStore) + } else { + regions, err = getTableIndexRegions(indexInfo, physicalIDs, tikvStore, splitStore) + } } else { // show table * region + if hasGlobalIndex { + physicalIDs = append([]int64{tb.Meta().ID}, physicalIDs...) + } regions, err = getTableRegions(tb, physicalIDs, tikvStore, splitStore) } if err != nil { From 617497366ac4f009880196b27fd5f4b7923d118e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20van=20Eeden?= Date: Thu, 17 Aug 2023 21:24:31 +0200 Subject: [PATCH 04/32] memory: Fix formatting of memory usage error for analyze table (#45696) close pingcap/tidb#45690 --- executor/BUILD.bazel | 1 + executor/analyze_utils.go | 2 +- executor/analyze_utils_test.go | 29 +++++++++++++++++++++++++++++ 3 files changed, 31 insertions(+), 1 deletion(-) create mode 100644 executor/analyze_utils_test.go diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index 855d4a05f949c..93a98773e3fc8 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -287,6 +287,7 @@ go_test( srcs = [ "adapter_test.go", "analyze_test.go", + "analyze_utils_test.go", "batch_point_get_test.go", "benchmark_test.go", "brie_test.go", diff --git a/executor/analyze_utils.go b/executor/analyze_utils.go index e7397df158a22..4bcdeca61b543 100644 --- a/executor/analyze_utils.go +++ b/executor/analyze_utils.go @@ -52,7 +52,7 @@ func getAnalyzePanicErr(r interface{}) error { return errAnalyzeOOM } if strings.Contains(msg, memory.PanicMemoryExceedWarnMsg) { - return errors.Errorf(msg, errAnalyzeOOM) + return errors.Errorf("%s, %s", msg, errAnalyzeOOM) } } if err, ok := r.(error); ok { diff --git a/executor/analyze_utils_test.go b/executor/analyze_utils_test.go new file mode 100644 index 0000000000000..f409fa66c79de --- /dev/null +++ b/executor/analyze_utils_test.go @@ -0,0 +1,29 @@ +// 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 executor + +import ( + "fmt" + "testing" + + "github.com/pingcap/tidb/util/memory" + "github.com/stretchr/testify/require" +) + +// https://github.com/pingcap/tidb/issues/45690 +func TestGetAnalyzePanicErr(t *testing.T) { + errMsg := fmt.Sprintf("%s", getAnalyzePanicErr(memory.PanicMemoryExceedWarnMsg)) + require.NotContains(t, errMsg, `%!(EXTRA`) +} From 6188778732dee8d5cabf9fcfdf5428a115576581 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Fri, 18 Aug 2023 10:41:02 +0800 Subject: [PATCH 05/32] lightning: add function about write-then-read to external engine (#46157) ref pingcap/tidb#45719 --- br/pkg/lightning/backend/external/BUILD.bazel | 7 +- br/pkg/lightning/backend/external/engine.go | 60 ++++++ .../lightning/backend/external/engine_test.go | 109 +++++++++++ br/pkg/lightning/backend/external/file.go | 78 +------- br/pkg/lightning/backend/external/iter.go | 4 + br/pkg/lightning/backend/external/util.go | 120 ++++++++++++ .../lightning/backend/external/util_test.go | 173 ++++++++++++++++++ br/pkg/lightning/backend/external/writer.go | 22 ++- .../lightning/backend/external/writer_test.go | 8 +- 9 files changed, 490 insertions(+), 91 deletions(-) create mode 100644 br/pkg/lightning/backend/external/engine.go create mode 100644 br/pkg/lightning/backend/external/engine_test.go create mode 100644 br/pkg/lightning/backend/external/util.go create mode 100644 br/pkg/lightning/backend/external/util_test.go diff --git a/br/pkg/lightning/backend/external/BUILD.bazel b/br/pkg/lightning/backend/external/BUILD.bazel index 5db3d4eaffc8d..1bba1a5b87da7 100644 --- a/br/pkg/lightning/backend/external/BUILD.bazel +++ b/br/pkg/lightning/backend/external/BUILD.bazel @@ -5,10 +5,12 @@ go_library( srcs = [ "byte_reader.go", "codec.go", + "engine.go", "file.go", "iter.go", "kv_reader.go", "stat_reader.go", + "util.go", "writer.go", ], importpath = "github.com/pingcap/tidb/br/pkg/lightning/backend/external", @@ -21,6 +23,7 @@ go_library( "//br/pkg/membuf", "//br/pkg/storage", "//kv", + "//util/hack", "//util/logutil", "//util/mathutil", "//util/size", @@ -36,13 +39,15 @@ go_test( srcs = [ "byte_reader_test.go", "codec_test.go", + "engine_test.go", "file_test.go", "iter_test.go", + "util_test.go", "writer_test.go", ], embed = [":external"], flaky = True, - shard_count = 15, + shard_count = 19, deps = [ "//br/pkg/lightning/backend/kv", "//br/pkg/lightning/common", diff --git a/br/pkg/lightning/backend/external/engine.go b/br/pkg/lightning/backend/external/engine.go new file mode 100644 index 0000000000000..357978f237601 --- /dev/null +++ b/br/pkg/lightning/backend/external/engine.go @@ -0,0 +1,60 @@ +// 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 external + +import ( + "context" + "encoding/hex" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" +) + +// Engine stored sorted key/value pairs in an external storage. +type Engine struct { + storage storage.ExternalStorage + dataFiles []string + statsFiles []string +} + +func (e *Engine) createMergeIter(ctx context.Context, start kv.Key) (*MergeKVIter, error) { + logger := logutil.Logger(ctx) + + var offsets []uint64 + if len(e.statsFiles) == 0 { + offsets = make([]uint64, len(e.dataFiles)) + logger.Info("no stats files", + zap.String("startKey", hex.EncodeToString(start))) + } else { + offs, err := seekPropsOffsets(ctx, start, e.statsFiles, e.storage) + if err != nil { + return nil, errors.Trace(err) + } + offsets = offs + logger.Info("seek props offsets", + zap.Uint64s("offsets", offsets), + zap.String("startKey", hex.EncodeToString(start)), + zap.Strings("dataFiles", prettyFileNames(e.dataFiles)), + zap.Strings("statsFiles", prettyFileNames(e.statsFiles))) + } + iter, err := NewMergeKVIter(ctx, e.dataFiles, offsets, e.storage, 64*1024) + if err != nil { + return nil, errors.Trace(err) + } + return iter, nil +} diff --git a/br/pkg/lightning/backend/external/engine_test.go b/br/pkg/lightning/backend/external/engine_test.go new file mode 100644 index 0000000000000..b27cb8770d7c8 --- /dev/null +++ b/br/pkg/lightning/backend/external/engine_test.go @@ -0,0 +1,109 @@ +// 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 external + +import ( + "bytes" + "context" + "slices" + "testing" + "time" + + "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" + "github.com/pingcap/tidb/br/pkg/lightning/common" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/stretchr/testify/require" + "golang.org/x/exp/rand" +) + +func TestIter(t *testing.T) { + seed := time.Now().Unix() + rand.Seed(uint64(seed)) + t.Logf("seed: %d", seed) + + totalKV := 300 + kvPairs := make([]common.KvPair, totalKV) + for i := range kvPairs { + keyBuf := make([]byte, rand.Intn(10)+1) + rand.Read(keyBuf) + // make sure the key is unique + kvPairs[i].Key = append(keyBuf, byte(i/255), byte(i%255)) + valBuf := make([]byte, rand.Intn(10)+1) + rand.Read(valBuf) + kvPairs[i].Val = valBuf + } + + sortedKVPairs := make([]common.KvPair, totalKV) + copy(sortedKVPairs, kvPairs) + slices.SortFunc(sortedKVPairs, func(i, j common.KvPair) int { + return bytes.Compare(i.Key, j.Key) + }) + + ctx := context.Background() + store := storage.NewMemStorage() + + for i := 0; i < 3; i++ { + w := NewWriterBuilder(). + SetMemorySizeLimit(uint64(rand.Intn(100)+1)). + SetPropSizeDistance(uint64(rand.Intn(50)+1)). + SetPropKeysDistance(uint64(rand.Intn(10)+1)). + Build(store, i, "/subtask") + kvStart := i * 100 + kvEnd := (i + 1) * 100 + err := w.AppendRows(ctx, nil, kv.MakeRowsFromKvPairs(kvPairs[kvStart:kvEnd])) + require.NoError(t, err) + _, err = w.Close(ctx) + require.NoError(t, err) + } + + dataFiles, statFiles, err := GetAllFileNames(ctx, store, "/subtask") + require.NoError(t, err) + + engine := Engine{ + storage: store, + dataFiles: dataFiles, + statsFiles: statFiles, + } + iter, err := engine.createMergeIter(ctx, sortedKVPairs[0].Key) + require.NoError(t, err) + got := make([]common.KvPair, 0, totalKV) + for iter.Next() { + got = append(got, common.KvPair{ + Key: iter.Key(), + Val: iter.Value(), + }) + } + require.NoError(t, iter.Error()) + require.Equal(t, sortedKVPairs, got) + + pickStartIdx := rand.Intn(len(sortedKVPairs)) + startKey := sortedKVPairs[pickStartIdx].Key + iter, err = engine.createMergeIter(ctx, startKey) + require.NoError(t, err) + got = make([]common.KvPair, 0, totalKV) + for iter.Next() { + got = append(got, common.KvPair{ + Key: iter.Key(), + Val: iter.Value(), + }) + } + require.NoError(t, iter.Error()) + // got keys must be ascending + for i := 1; i < len(got); i++ { + require.True(t, bytes.Compare(got[i-1].Key, got[i].Key) < 0) + } + // the first key must be less than or equal to startKey + require.True(t, bytes.Compare(got[0].Key, startKey) <= 0) +} diff --git a/br/pkg/lightning/backend/external/file.go b/br/pkg/lightning/backend/external/file.go index 8d5c3dd2bf6af..685f0495e4a2b 100644 --- a/br/pkg/lightning/backend/external/file.go +++ b/br/pkg/lightning/backend/external/file.go @@ -17,9 +17,6 @@ package external import ( "context" "encoding/binary" - "path/filepath" - "strconv" - "strings" "github.com/pingcap/tidb/br/pkg/storage" ) @@ -117,77 +114,4 @@ func (s *KeyValueStore) Close() { } } -var statSuffix = filepath.Join("_stat", "0") - -// GetAllFileNames returns a FilePathHandle that contains all data file paths -// and a slice of stat file paths. -func GetAllFileNames( - ctx context.Context, - store storage.ExternalStorage, - subDir string, -) (FilePathHandle, []string, error) { - var dataFilePaths FilePathHandle - var stats []string - - err := store.WalkDir(ctx, - &storage.WalkOption{SubDir: subDir}, - func(path string, size int64) error { - if strings.HasSuffix(path, statSuffix) { - stats = append(stats, path) - } else { - dir, file := filepath.Split(path) - writerID, err := strconv.Atoi(filepath.Base(dir)) - if err != nil { - return err - } - seq, err := strconv.Atoi(file) - if err != nil { - return err - } - dataFilePaths.set(writerID, seq, path) - } - return nil - }) - if err != nil { - return dataFilePaths, nil, err - } - return dataFilePaths, stats, nil -} - -// FilePathHandle handles data file paths under a prefix path. -type FilePathHandle struct { - paths [][]string -} - -func (p *FilePathHandle) set(writerID, seq int, path string) { - if writerID >= len(p.paths) { - p.paths = append(p.paths, make([][]string, writerID-len(p.paths)+1)...) - } - if seq >= len(p.paths[writerID]) { - p.paths[writerID] = append(p.paths[writerID], make([]string, seq-len(p.paths[writerID])+1)...) - } - p.paths[writerID][seq] = path -} - -// Get returns the path of the data file with the given writerID and seq. -func (p *FilePathHandle) Get(writerID, seq int) string { - return p.paths[writerID][seq] -} - -// ForEach applies the given function to each data file path. -func (p *FilePathHandle) ForEach(f func(writerID, seq int, path string)) { - for writerID, paths := range p.paths { - for seq, path := range paths { - f(writerID, seq, path) - } - } -} - -// FlatSlice returns a flat slice of all data file paths. -func (p *FilePathHandle) FlatSlice() []string { - var paths []string - p.ForEach(func(writerID, seq int, path string) { - paths = append(paths, path) - }) - return paths -} +const statSuffix = "_stat" diff --git a/br/pkg/lightning/backend/external/iter.go b/br/pkg/lightning/backend/external/iter.go index f892783fb6a27..1a7874d4baa54 100644 --- a/br/pkg/lightning/backend/external/iter.go +++ b/br/pkg/lightning/backend/external/iter.go @@ -378,6 +378,10 @@ func (i *MergePropIter) prop() *rangeProperty { return i.iter.curr } +func (i *MergePropIter) readerIndex() int { + return i.iter.lastReaderIdx +} + // Close closes the iterator. func (i *MergePropIter) Close() error { return i.iter.close() diff --git a/br/pkg/lightning/backend/external/util.go b/br/pkg/lightning/backend/external/util.go new file mode 100644 index 0000000000000..505ef41660ee2 --- /dev/null +++ b/br/pkg/lightning/backend/external/util.go @@ -0,0 +1,120 @@ +// 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 external + +import ( + "bytes" + "context" + "fmt" + "path/filepath" + "sort" + "strings" + + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" +) + +// prettyFileNames removes the directory prefix except the last level from the +// file names. +func prettyFileNames(files []string) []string { + names := make([]string, 0, len(files)) + for _, f := range files { + dir, file := filepath.Split(f) + names = append(names, fmt.Sprintf("%s/%s", filepath.Base(dir), file)) + } + return names +} + +// seekPropsOffsets seeks the statistic files to find the largest offset of +// sorted data file offsets such that the key at offset is less than or equal to +// the given start key. +func seekPropsOffsets( + ctx context.Context, + start kv.Key, + paths []string, + exStorage storage.ExternalStorage, +) ([]uint64, error) { + iter, err := NewMergePropIter(ctx, paths, exStorage) + if err != nil { + return nil, err + } + logger := logutil.Logger(ctx) + defer func() { + if err := iter.Close(); err != nil { + logger.Warn("failed to close merge prop iterator", zap.Error(err)) + } + }() + offsets := make([]uint64, len(paths)) + moved := false + for iter.Next() { + p := iter.prop() + propKey := kv.Key(p.key) + if propKey.Cmp(start) > 0 { + if !moved { + return nil, fmt.Errorf("start key %s is too small for stat files %v", + start.String(), + paths, + ) + } + return offsets, nil + } + moved = true + offsets[iter.readerIndex()] = p.offset + } + if iter.Error() != nil { + return nil, iter.Error() + } + return offsets, nil +} + +// GetAllFileNames returns data file paths and stat file paths. Both paths are +// sorted. +func GetAllFileNames( + ctx context.Context, + store storage.ExternalStorage, + subDir string, +) ([]string, []string, error) { + var data []string + var stats []string + + err := store.WalkDir(ctx, + &storage.WalkOption{SubDir: subDir}, + func(path string, size int64) error { + // path example: /subtask/0_stat/0 + + // extract the parent dir + bs := hack.Slice(path) + lastIdx := bytes.LastIndexByte(bs, '/') + secondLastIdx := bytes.LastIndexByte(bs[:lastIdx], '/') + parentDir := path[secondLastIdx+1 : lastIdx] + + if strings.HasSuffix(parentDir, statSuffix) { + stats = append(stats, path) + } else { + data = append(data, path) + } + return nil + }) + if err != nil { + return nil, nil, err + } + // in case the external storage does not guarantee the order of walk + sort.Strings(data) + sort.Strings(stats) + return data, stats, nil +} diff --git a/br/pkg/lightning/backend/external/util_test.go b/br/pkg/lightning/backend/external/util_test.go new file mode 100644 index 0000000000000..622872a6bfe27 --- /dev/null +++ b/br/pkg/lightning/backend/external/util_test.go @@ -0,0 +1,173 @@ +// 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 external + +import ( + "context" + "testing" + + "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" + "github.com/pingcap/tidb/br/pkg/lightning/common" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/stretchr/testify/require" +) + +func TestPrettyFileNames(t *testing.T) { + filenames := []string{ + "/tmp/br/backup/1/1_1.sst", + "/tmp/br/2/1_2.sst", + "/tmp/123/1/1_3", + } + expected := []string{ + "1/1_1.sst", + "2/1_2.sst", + "1/1_3", + } + require.Equal(t, expected, prettyFileNames(filenames)) +} + +func TestSeekPropsOffsets(t *testing.T) { + ctx := context.Background() + store := storage.NewMemStorage() + + rc1 := &rangePropertiesCollector{ + props: []*rangeProperty{ + { + key: []byte("key1"), + offset: 10, + }, + { + key: []byte("key3"), + offset: 30, + }, + { + key: []byte("key5"), + offset: 50, + }, + }, + } + file1 := "/test1" + w1, err := store.Create(ctx, file1, nil) + require.NoError(t, err) + _, err = w1.Write(ctx, rc1.encode()) + require.NoError(t, err) + err = w1.Close(ctx) + require.NoError(t, err) + + rc2 := &rangePropertiesCollector{ + props: []*rangeProperty{ + { + key: []byte("key2"), + offset: 20, + }, + { + key: []byte("key4"), + offset: 40, + }, + }, + } + file2 := "/test2" + w2, err := store.Create(ctx, file2, nil) + require.NoError(t, err) + _, err = w2.Write(ctx, rc2.encode()) + require.NoError(t, err) + err = w2.Close(ctx) + require.NoError(t, err) + + got, err := seekPropsOffsets(ctx, []byte("key2.5"), []string{file1, file2}, store) + require.NoError(t, err) + require.Equal(t, []uint64{10, 20}, got) + got, err = seekPropsOffsets(ctx, []byte("key3"), []string{file1, file2}, store) + require.NoError(t, err) + require.Equal(t, []uint64{30, 20}, got) + got, err = seekPropsOffsets(ctx, []byte("key0"), []string{file1, file2}, store) + require.ErrorContains(t, err, "start key 6b657930 is too small for stat files [/test1 /test2]") + got, err = seekPropsOffsets(ctx, []byte("key1"), []string{file1, file2}, store) + require.NoError(t, err) + require.Equal(t, []uint64{10, 0}, got) + got, err = seekPropsOffsets(ctx, []byte("key999"), []string{file1, file2}, store) + require.NoError(t, err) + require.Equal(t, []uint64{50, 40}, got) + + file3 := "/test3" + w3, err := store.Create(ctx, file3, nil) + require.NoError(t, err) + err = w3.Close(ctx) + + file4 := "/test4" + w4, err := store.Create(ctx, file4, nil) + require.NoError(t, err) + _, err = w4.Write(ctx, rc1.encode()) + require.NoError(t, err) + err = w4.Close(ctx) + require.NoError(t, err) + got, err = seekPropsOffsets(ctx, []byte("key3"), []string{file1, file2, file3, file4}, store) + require.NoError(t, err) + require.Equal(t, []uint64{30, 20, 0, 30}, got) +} + +func TestGetAllFileNames(t *testing.T) { + ctx := context.Background() + store := storage.NewMemStorage() + w := NewWriterBuilder(). + SetMemorySizeLimit(20). + SetPropSizeDistance(5). + SetPropKeysDistance(3). + Build(store, 0, "/subtask") + kvPairs := make([]common.KvPair, 0, 30) + for i := 0; i < 30; i++ { + kvPairs = append(kvPairs, common.KvPair{ + Key: []byte{byte(i)}, + Val: []byte{byte(i)}, + }) + } + err := w.AppendRows(ctx, nil, kv.MakeRowsFromKvPairs(kvPairs)) + require.NoError(t, err) + _, err = w.Close(ctx) + require.NoError(t, err) + + w2 := NewWriterBuilder(). + SetMemorySizeLimit(20). + SetPropSizeDistance(5). + SetPropKeysDistance(3). + Build(store, 3, "/subtask") + err = w2.AppendRows(ctx, nil, kv.MakeRowsFromKvPairs(kvPairs)) + require.NoError(t, err) + _, err = w2.Close(ctx) + require.NoError(t, err) + + w3 := NewWriterBuilder(). + SetMemorySizeLimit(20). + SetPropSizeDistance(5). + SetPropKeysDistance(3). + Build(store, 12, "/subtask") + err = w3.AppendRows(ctx, nil, kv.MakeRowsFromKvPairs(kvPairs)) + require.NoError(t, err) + _, err = w3.Close(ctx) + require.NoError(t, err) + + dataFiles, statFiles, err := GetAllFileNames(ctx, store, "/subtask") + require.NoError(t, err) + require.Equal(t, []string{ + "/subtask/0_stat/0", "/subtask/0_stat/1", "/subtask/0_stat/2", + "/subtask/12_stat/0", "/subtask/12_stat/1", "/subtask/12_stat/2", + "/subtask/3_stat/0", "/subtask/3_stat/1", "/subtask/3_stat/2", + }, statFiles) + require.Equal(t, []string{ + "/subtask/0/0", "/subtask/0/1", "/subtask/0/2", + "/subtask/12/0", "/subtask/12/1", "/subtask/12/2", + "/subtask/3/0", "/subtask/3/1", "/subtask/3/2", + }, dataFiles) +} diff --git a/br/pkg/lightning/backend/external/writer.go b/br/pkg/lightning/backend/external/writer.go index 79c6349953ba1..6fac1f3ea5371 100644 --- a/br/pkg/lightning/backend/external/writer.go +++ b/br/pkg/lightning/backend/external/writer.go @@ -19,11 +19,10 @@ import ( "context" "encoding/hex" "path/filepath" + "slices" "strconv" "time" - "slices" - "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/backend/encode" @@ -95,7 +94,9 @@ func NewWriterBuilder() *WriterBuilder { } } -// SetMemorySizeLimit sets the memory size limit of the writer. +// SetMemorySizeLimit sets the memory size limit of the writer. When accumulated +// data size exceeds this limit, the writer will flush data as a file to external +// storage. func (b *WriterBuilder) SetMemorySizeLimit(size uint64) *WriterBuilder { b.memSizeLimit = size return b @@ -131,16 +132,18 @@ func (b *WriterBuilder) SetBufferPool(bufferPool *membuf.Pool) *WriterBuilder { return b } -// Build builds a new Writer. +// Build builds a new Writer. The files writer will create are under the prefix +// of "{prefix}/{writerID}". func (b *WriterBuilder) Build( store storage.ExternalStorage, writerID int, - filenamePrefix string, + prefix string, ) *Writer { bp := b.bufferPool if bp == nil { bp = membuf.NewPool() } + filenamePrefix := filepath.Join(prefix, strconv.Itoa(writerID)) return &Writer{ rc: &rangePropertiesCollector{ props: make([]*rangeProperty, 0, 1024), @@ -262,6 +265,7 @@ func (w *Writer) flushKVs(ctx context.Context) (err error) { return nil } + logger := logutil.Logger(ctx) dataWriter, statWriter, err := w.createStorageWriter(ctx) if err != nil { return err @@ -277,16 +281,16 @@ func (w *Writer) flushKVs(ctx context.Context) (err error) { return } if err1 != nil { - logutil.Logger(ctx).Error("close data writer failed", zap.Error(err)) + logger.Error("close data writer failed", zap.Error(err)) err = err1 return } if err2 != nil { - logutil.Logger(ctx).Error("close stat writer failed", zap.Error(err)) + logger.Error("close stat writer failed", zap.Error(err)) err = err2 return } - logutil.Logger(ctx).Info("flush kv", + logger.Info("flush kv", zap.Duration("time", time.Since(ts)), zap.Uint64("bytes", savedBytes), zap.Any("rate", float64(savedBytes)/1024.0/1024.0/time.Since(ts).Seconds())) @@ -332,7 +336,7 @@ func (w *Writer) createStorageWriter(ctx context.Context) (data, stats storage.E if err != nil { return nil, nil, err } - statPath := filepath.Join(w.filenamePrefix+"_stat", strconv.Itoa(w.currentSeq)) + 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 diff --git a/br/pkg/lightning/backend/external/writer_test.go b/br/pkg/lightning/backend/external/writer_test.go index b540b82898516..d12afc3562ff9 100644 --- a/br/pkg/lightning/backend/external/writer_test.go +++ b/br/pkg/lightning/backend/external/writer_test.go @@ -67,7 +67,7 @@ func TestWriter(t *testing.T) { }) bufSize := rand.Intn(100) + 1 - kvReader, err := newKVReader(ctx, "/test/0", memStore, 0, bufSize) + kvReader, err := newKVReader(ctx, "/test/0/0", memStore, 0, bufSize) require.NoError(t, err) for i := 0; i < kvCnt; i++ { key, value, err := kvReader.nextKV() @@ -78,7 +78,7 @@ func TestWriter(t *testing.T) { _, _, err = kvReader.nextKV() require.Equal(t, io.EOF, err) - statReader, err := newStatsReader(ctx, memStore, "/test_stat/0", bufSize) + statReader, err := newStatsReader(ctx, memStore, "/test/0_stat/0", bufSize) require.NoError(t, err) var keyCnt uint64 = 0 @@ -136,7 +136,7 @@ func TestWriterFlushMultiFileNames(t *testing.T) { require.Len(t, dataFiles, 4) require.Len(t, statFiles, 4) for i := 0; i < 4; i++ { - require.Equal(t, dataFiles[i], fmt.Sprintf("/test/%d", i)) - require.Equal(t, statFiles[i], fmt.Sprintf("/test_stat/%d", i)) + require.Equal(t, dataFiles[i], fmt.Sprintf("/test/0/%d", i)) + require.Equal(t, statFiles[i], fmt.Sprintf("/test/0_stat/%d", i)) } } From 0dad9f27fcf9bccdcbe82c88f0dde93447e5d9d3 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Fri, 18 Aug 2023 11:47:01 +0800 Subject: [PATCH 06/32] *: fix the duplicate entry error when using BR to restore a NONCLUSTERED AUTO_ID_CACHE=1 table (#46127) close pingcap/tidb#46093 --- br/pkg/backup/client.go | 14 +++++++++++ br/tests/br_autoid/run.sh | 51 +++++++++++++++++++++++++++++++++++++++ br/tests/run_group.sh | 2 +- ddl/ddl_api.go | 6 +++++ parser/model/model.go | 31 ++++++++++++++++++------ 5 files changed, 95 insertions(+), 9 deletions(-) create mode 100644 br/tests/br_autoid/run.sh diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index c18648acc29da..edbe3efd0ae98 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -633,6 +633,20 @@ func BuildBackupSchemas( default: if tableInfo.SepAutoInc() { globalAutoID, err = autoIDAccess.IncrementID(tableInfo.Version).Get() + // For a nonclustered table with auto_increment column, both auto_increment_id and _tidb_rowid are required. + // See also https://github.com/pingcap/tidb/issues/46093 + if rowID, err1 := autoIDAccess.RowID().Get(); err1 == nil { + tableInfo.AutoIncIDExtra = rowID + 1 + } else { + // It is possible that the rowid meta key does not exist (i.e. table have auto_increment_id but no _rowid), + // so err1 != nil might be expected. + if globalAutoID == 0 { + // When both auto_increment_id and _rowid are missing, it must be something wrong. + return errors.Trace(err1) + } + // Print a warning in other scenes, should it be a INFO log? + log.Warn("get rowid error", zap.Error(err1)) + } } else { globalAutoID, err = autoIDAccess.RowID().Get() } diff --git a/br/tests/br_autoid/run.sh b/br/tests/br_autoid/run.sh new file mode 100644 index 0000000000000..af0ee46c7f582 --- /dev/null +++ b/br/tests/br_autoid/run.sh @@ -0,0 +1,51 @@ +#!/bin/sh +# +# 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. + +set -eu +DB="$TEST_NAME" + +run_sql "create database if not exists ${DB}" +run_sql "create table $DB.issue46093 (a int primary key nonclustered auto_increment, b int) auto_id_cache = 1;" +run_sql "insert into $DB.issue46093 (b) values (1), (2), (3);" +run_sql "show table $DB.issue46093 next_row_id;" +check_contains "NEXT_GLOBAL_ROW_ID: 30001" +check_contains "NEXT_GLOBAL_ROW_ID: 4" + +run_sql "backup table $DB.issue46093 to 'local://$TEST_DIR/$DB'"; +run_sql "drop table $DB.issue46093;" +run_sql "restore table $DB.issue46093 from 'local://$TEST_DIR/$DB';" + +run_sql "show table $DB.issue46093 next_row_id;" +check_contains "NEXT_GLOBAL_ROW_ID: 30001" +check_contains "NEXT_GLOBAL_ROW_ID: 4001" +run_sql "insert into $DB.issue46093 (b) values (4), (5), (6);" +run_sql "insert into $DB.issue46093 (b) values (7), (8), (9);" +run_sql "select * from $DB.issue46093;" +check_contains "a: 1" +check_contains "a: 2" +check_contains "a: 3" +check_contains "a: 4001" +check_contains "a: 4002" +check_contains "a: 4003" +check_contains "a: 4004" +check_contains "a: 4005" +check_contains "a: 4006" +check_contains "b: 4" +check_contains "b: 5" +check_contains "b: 6" +check_contains "b: 7" +check_contains "b: 8" +check_contains "b: 9" diff --git a/br/tests/run_group.sh b/br/tests/run_group.sh index ced89dd6bf360..58fe387d4be6a 100755 --- a/br/tests/run_group.sh +++ b/br/tests/run_group.sh @@ -21,7 +21,7 @@ mkdir $COV_DIR declare -A groups groups=( ["G00"]="br_300_small_tables br_backup_empty br_backup_version br_cache_table br_case_sensitive br_charset_gbk br_check_new_collocation_enable" - ["G01"]="br_crypter2 br_db br_db_online br_db_online_newkv br_db_skip br_debug_meta br_ebs br_foreign_key br_full" + ["G01"]="br_autoid br_crypter2 br_db br_db_online br_db_online_newkv br_db_skip br_debug_meta br_ebs br_foreign_key br_full" ["G02"]="br_full_cluster_restore br_full_ddl br_full_index br_gcs br_history" ["G03"]='br_incompatible_tidb_config br_incremental br_incremental_ddl br_incremental_index' ["G04"]='br_incremental_only_ddl br_incremental_same_table br_insert_after_restore br_key_locked br_log_test br_move_backup br_mv_index br_other br_partition_add_index' diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 6bfb0837a306f..2d72c7bd2900c 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2628,6 +2628,12 @@ func (d *ddl) createTableWithInfoPost( return errors.Trace(err) } } + // For issue https://github.com/pingcap/tidb/issues/46093 + if tbInfo.AutoIncIDExtra != 0 { + if err = d.handleAutoIncID(tbInfo, schemaID, tbInfo.AutoIncIDExtra-1, autoid.RowIDAllocType); err != nil { + return errors.Trace(err) + } + } if tbInfo.AutoRandID > 1 { // Default tableAutoRandID base is 0. // If the first ID is expected to greater than 1, we need to do rebase. diff --git a/parser/model/model.go b/parser/model/model.go index 0059cc60e05bf..64cda27eb4d53 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -458,14 +458,29 @@ type TableInfo struct { // 1 for the clustered index created > 5.0.0 RC. CommonHandleVersion uint16 `json:"common_handle_version"` - Comment string `json:"comment"` - AutoIncID int64 `json:"auto_inc_id"` - AutoIdCache int64 `json:"auto_id_cache"` //nolint:revive - AutoRandID int64 `json:"auto_rand_id"` - MaxColumnID int64 `json:"max_col_id"` - MaxIndexID int64 `json:"max_idx_id"` - MaxForeignKeyID int64 `json:"max_fk_id"` - MaxConstraintID int64 `json:"max_cst_id"` + Comment string `json:"comment"` + AutoIncID int64 `json:"auto_inc_id"` + + // Only used by BR when: + // 1. SepAutoInc() is true + // 2. The table is nonclustered and has auto_increment column. + // In that case, both auto_increment_id and tidb_rowid need to be backup & recover. + // See also https://github.com/pingcap/tidb/issues/46093 + // + // It should have been named TiDBRowID, but for historial reasons, we do not use separate meta key for _tidb_rowid and auto_increment_id, + // and field `AutoIncID` is used to serve both _tidb_rowid and auto_increment_id. + // If we introduce a TiDBRowID here, it could make furthur misunderstanding: + // in most cases, AutoIncID is _tidb_rowid and TiDBRowID is null + // but in some cases, AutoIncID is auto_increment_id and TiDBRowID is _tidb_rowid + // So let's just use another name AutoIncIDExtra to avoid misconception. + AutoIncIDExtra int64 `json:"auto_inc_id_extra,omitempty"` + + AutoIdCache int64 `json:"auto_id_cache"` //nolint:revive + AutoRandID int64 `json:"auto_rand_id"` + MaxColumnID int64 `json:"max_col_id"` + MaxIndexID int64 `json:"max_idx_id"` + MaxForeignKeyID int64 `json:"max_fk_id"` + MaxConstraintID int64 `json:"max_cst_id"` // UpdateTS is used to record the timestamp of updating the table's schema information. // These changing schema operations don't include 'truncate table' and 'rename table'. UpdateTS uint64 `json:"update_timestamp"` From deba2b08111d654e621c7e15a5cef044cdf3c602 Mon Sep 17 00:00:00 2001 From: Yasuo Honda Date: Fri, 18 Aug 2023 13:48:31 +0900 Subject: [PATCH 07/32] expression: Replace `initialzied` with `initialized` (#46208) --- expression/builtin_grouping.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/expression/builtin_grouping.go b/expression/builtin_grouping.go index 005db3d24efc1..a08f3a713198b 100644 --- a/expression/builtin_grouping.go +++ b/expression/builtin_grouping.go @@ -229,7 +229,7 @@ func (b *BuiltinGroupingImplSig) grouping(groupingID uint64) int64 { // evalInt evals a builtinGroupingSig. func (b *BuiltinGroupingImplSig) evalInt(row chunk.Row) (int64, bool, error) { if !b.isMetaInited { - return 0, false, errors.Errorf("Meta data is not initialzied") + return 0, false, errors.Errorf("Meta data is not initialized") } // grouping function should be rewritten from raw column ref to built gid column and groupingMarks meta. groupingID, isNull, err := b.args[0].EvalInt(b.ctx, row) @@ -261,7 +261,7 @@ func (b *BuiltinGroupingImplSig) groupingVec(groupingIds *chunk.Column, rowNum i func (b *BuiltinGroupingImplSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error { if !b.isMetaInited { - return errors.Errorf("Meta data is not initialzied") + return errors.Errorf("Meta data is not initialized") } rowNum := input.NumRows() From 2ac191a1379c35ebae0b4297a3e44142b08b0080 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=B1=B1=E5=B2=9A?= <36239017+YuJuncen@users.noreply.github.com> Date: Fri, 18 Aug 2023 13:45:31 +0800 Subject: [PATCH 08/32] snap_restore: added retry for recovery (#46094) close pingcap/tidb#46181 --- br/pkg/restore/data.go | 96 ++++++++++++++++++++++++++++++++++++++--- br/pkg/utils/backoff.go | 4 ++ br/pkg/utils/retry.go | 27 ++++++++++-- 3 files changed, 116 insertions(+), 11 deletions(-) diff --git a/br/pkg/restore/data.go b/br/pkg/restore/data.go index 061c3114980ba..acbe595b59007 100644 --- a/br/pkg/restore/data.go +++ b/br/pkg/restore/data.go @@ -27,6 +27,75 @@ import ( "google.golang.org/grpc/backoff" ) +type RecoveryStage int + +const ( + StageUnknown RecoveryStage = iota + StageCollectingMeta + StageMakingRecoveryPlan + StageResetPDAllocateID + StageRecovering + StageFlashback +) + +func (s RecoveryStage) String() string { + switch s { + case StageCollectingMeta: + return "collecting meta" + case StageMakingRecoveryPlan: + return "making recovery plan" + case StageResetPDAllocateID: + return "resetting PD allocate ID" + case StageRecovering: + return "recovering" + case StageFlashback: + return "flashback" + default: + return "unknown" + } +} + +type recoveryError struct { + error + atStage RecoveryStage +} + +func FailedAt(err error) RecoveryStage { + if rerr, ok := err.(recoveryError); ok { + return rerr.atStage + } + return StageUnknown +} + +type recoveryBackoffer struct { + state utils.RetryState +} + +func newRecoveryBackoffer() *recoveryBackoffer { + return &recoveryBackoffer{ + state: utils.InitialRetryState(16, 30*time.Second, 4*time.Minute), + } +} + +func (bo *recoveryBackoffer) NextBackoff(err error) time.Duration { + s := FailedAt(err) + switch s { + case StageCollectingMeta, StageMakingRecoveryPlan, StageResetPDAllocateID, StageRecovering: + log.Info("Recovery data retrying.", zap.Error(err), zap.Stringer("stage", s)) + return bo.state.ExponentialBackoff() + case StageFlashback: + log.Info("Giving up retry for flashback stage.", zap.Error(err), zap.Stringer("stage", s)) + bo.state.GiveUp() + return 0 + } + log.Warn("unknown stage of backing off.", zap.Int("val", int(s))) + return bo.state.ExponentialBackoff() +} + +func (bo *recoveryBackoffer) Attempt() int { + return bo.state.Attempt() +} + // RecoverData recover the tikv cluster // 1. read all meta data from tikvs // 2. make recovery plan and then recovery max allocate ID firstly @@ -35,39 +104,52 @@ import ( // 5. prepare the flashback // 6. flashback to resolveTS func RecoverData(ctx context.Context, resolveTS uint64, allStores []*metapb.Store, mgr *conn.Mgr, progress glue.Progress, restoreTS uint64, concurrency uint32) (int, error) { + // Roughly handle the case that some TiKVs are rebooted during making plan. + // Generally, retry the whole procedure will be fine for most cases. But perhaps we can do finer-grained retry, + // say, we may reuse the recovery plan, and probably no need to rebase PD allocation ID once we have done it. + return utils.WithRetryV2(ctx, newRecoveryBackoffer(), func(ctx context.Context) (int, error) { + return doRecoveryData(ctx, resolveTS, allStores, mgr, progress, restoreTS, concurrency) + }) +} + +func doRecoveryData(ctx context.Context, resolveTS uint64, allStores []*metapb.Store, mgr *conn.Mgr, progress glue.Progress, restoreTS uint64, concurrency uint32) (int, error) { + var cancel context.CancelFunc + ctx, cancel = context.WithCancel(ctx) + defer cancel() + var recovery = NewRecovery(allStores, mgr, progress, concurrency) if err := recovery.ReadRegionMeta(ctx); err != nil { - return 0, errors.Trace(err) + return 0, recoveryError{error: err, atStage: StageCollectingMeta} } totalRegions := recovery.GetTotalRegions() if err := recovery.MakeRecoveryPlan(); err != nil { - return totalRegions, errors.Trace(err) + return totalRegions, recoveryError{error: err, atStage: StageMakingRecoveryPlan} } log.Info("recover the alloc id to pd", zap.Uint64("max alloc id", recovery.MaxAllocID)) if err := recovery.mgr.RecoverBaseAllocID(ctx, recovery.MaxAllocID); err != nil { - return totalRegions, errors.Trace(err) + return totalRegions, recoveryError{error: err, atStage: StageResetPDAllocateID} } // Once TiKV shuts down and reboot then, it may be left with no leader because of the recovery mode. // This wathcher will retrigger `RecoveryRegions` for those stores. recovery.SpawnTiKVShutDownWatchers(ctx) if err := recovery.RecoverRegions(ctx); err != nil { - return totalRegions, errors.Trace(err) + return totalRegions, recoveryError{error: err, atStage: StageRecovering} } if err := recovery.WaitApply(ctx); err != nil { - return totalRegions, errors.Trace(err) + return totalRegions, recoveryError{error: err, atStage: StageRecovering} } if err := recovery.PrepareFlashbackToVersion(ctx, resolveTS, restoreTS-1); err != nil { - return totalRegions, errors.Trace(err) + return totalRegions, recoveryError{error: err, atStage: StageFlashback} } if err := recovery.FlashbackToVersion(ctx, resolveTS, restoreTS); err != nil { - return totalRegions, errors.Trace(err) + return totalRegions, recoveryError{error: err, atStage: StageFlashback} } return totalRegions, nil diff --git a/br/pkg/utils/backoff.go b/br/pkg/utils/backoff.go index 02d21994118e6..368c11b7f0b6a 100644 --- a/br/pkg/utils/backoff.go +++ b/br/pkg/utils/backoff.go @@ -72,6 +72,10 @@ func (rs *RetryState) ExponentialBackoff() time.Duration { return backoff } +func (rs *RetryState) GiveUp() { + rs.retryTimes = rs.maxRetry +} + // InitialRetryState make the initial state for retrying. func InitialRetryState(maxRetryTimes int, initialBackoff, maxBackoff time.Duration) RetryState { return RetryState{ diff --git a/br/pkg/utils/retry.go b/br/pkg/utils/retry.go index 1ffaaaadbc289..b4ab0437cf651 100644 --- a/br/pkg/utils/retry.go +++ b/br/pkg/utils/retry.go @@ -34,6 +34,8 @@ var retryableServerError = []string{ // RetryableFunc presents a retryable operation. type RetryableFunc func() error +type RetryableFuncV2[T any] func(context.Context) (T, error) + // Backoffer implements a backoff policy for retrying operations. type Backoffer interface { // NextBackoff returns a duration to wait before retrying again @@ -51,20 +53,37 @@ func WithRetry( retryableFunc RetryableFunc, backoffer Backoffer, ) error { + _, err := WithRetryV2[struct{}](ctx, backoffer, func(ctx context.Context) (struct{}, error) { + innerErr := retryableFunc() + return struct{}{}, innerErr + }) + return err +} + +// WithRetryV2 retries a given operation with a backoff policy. +// +// Returns the returned value if `retryableFunc` succeeded at least once. Otherwise, returns a +// multierr that containing all errors encountered. +// Comparing with `WithRetry`, this function reordered the argument order and supports catching the return value. +func WithRetryV2[T any]( + ctx context.Context, + backoffer Backoffer, + fn RetryableFuncV2[T], +) (T, error) { var allErrors error for backoffer.Attempt() > 0 { - err := retryableFunc() + res, err := fn(ctx) if err == nil { - return nil + return res, nil } allErrors = multierr.Append(allErrors, err) select { case <-ctx.Done(): - return allErrors // nolint:wrapcheck + return *new(T), allErrors case <-time.After(backoffer.NextBackoff(err)): } } - return allErrors // nolint:wrapcheck + return *new(T), allErrors // nolint:wrapcheck } // MessageIsRetryableStorageError checks whether the message returning from TiKV is retryable ExternalStorageError. From 592d52ed8e19e5553d047a924e53726612476f14 Mon Sep 17 00:00:00 2001 From: Yongbo Jiang Date: Fri, 18 Aug 2023 15:00:30 +0800 Subject: [PATCH 09/32] domain: initResourceGroupsController after fetching ServerID (#46220) close pingcap/tidb#46200 --- domain/domain.go | 10 ++++++---- domain/runaway.go | 4 ++-- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index a75b340d18f67..115774c153531 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1173,10 +1173,6 @@ func (do *Domain) Init( if err != nil { return err } - err = do.initResourceGroupsController(ctx, pdCli) - if err != nil { - return err - } do.globalCfgSyncer = globalconfigsync.NewGlobalConfigSyncer(pdCli) err = do.ddl.SchemaSyncer().Init(ctx) if err != nil { @@ -1209,6 +1205,12 @@ func (do *Domain) Init( do.connIDAllocator = globalconn.NewSimpleAllocator() } + // should put `initResourceGroupsController` after fetching server ID + err = do.initResourceGroupsController(ctx, pdCli, do.ServerID()) + if err != nil { + return err + } + startReloadTime := time.Now() // step 3: domain reload the infoSchema. err = do.Reload() diff --git a/domain/runaway.go b/domain/runaway.go index 220f763ee17c0..0840b51f1fdce 100644 --- a/domain/runaway.go +++ b/domain/runaway.go @@ -392,14 +392,14 @@ func (do *Domain) execRestrictedSQL(sql string, params []interface{}) ([]chunk.R return r, err } -func (do *Domain) initResourceGroupsController(ctx context.Context, pdClient pd.Client) error { +func (do *Domain) initResourceGroupsController(ctx context.Context, pdClient pd.Client, uniqueID uint64) error { if pdClient == nil { logutil.BgLogger().Warn("cannot setup up resource controller, not using tikv storage") // return nil as unistore doesn't support it return nil } - control, err := rmclient.NewResourceGroupController(ctx, do.ServerID(), pdClient, nil, rmclient.WithMaxWaitDuration(resourcegroup.MaxWaitDuration)) + control, err := rmclient.NewResourceGroupController(ctx, uniqueID, pdClient, nil, rmclient.WithMaxWaitDuration(resourcegroup.MaxWaitDuration)) if err != nil { return err } From 883e88a64568c3cb4ee9da9286b45f05bc1dc744 Mon Sep 17 00:00:00 2001 From: GMHDBJD <35025882+GMHDBJD@users.noreply.github.com> Date: Fri, 18 Aug 2023 15:36:32 +0800 Subject: [PATCH 10/32] distribute framework: restrict the task state transform rule (#45932) ref pingcap/tidb#41495 --- disttask/framework/dispatcher/BUILD.bazel | 2 +- disttask/framework/dispatcher/dispatcher.go | 65 ++++++++++++++++++- .../framework/dispatcher/dispatcher_test.go | 22 +++++++ disttask/framework/proto/task.go | 1 + disttask/framework/storage/table_test.go | 12 ++-- disttask/framework/storage/task_table.go | 6 +- 6 files changed, 97 insertions(+), 11 deletions(-) diff --git a/disttask/framework/dispatcher/BUILD.bazel b/disttask/framework/dispatcher/BUILD.bazel index d9a6a23857790..5675ebaaac738 100644 --- a/disttask/framework/dispatcher/BUILD.bazel +++ b/disttask/framework/dispatcher/BUILD.bazel @@ -40,7 +40,7 @@ go_test( embed = [":dispatcher"], flaky = True, race = "on", - shard_count = 7, + shard_count = 8, deps = [ "//disttask/framework/proto", "//disttask/framework/storage", diff --git a/disttask/framework/dispatcher/dispatcher.go b/disttask/framework/dispatcher/dispatcher.go index e2c813a687755..b9753d09533e7 100644 --- a/disttask/framework/dispatcher/dispatcher.go +++ b/disttask/framework/dispatcher/dispatcher.go @@ -215,8 +215,12 @@ func (d *dispatcher) handleRunning() error { func (d *dispatcher) updateTask(taskState string, newSubTasks []*proto.Subtask, retryTimes int) (err error) { prevState := d.task.State d.task.State = taskState + if !VerifyTaskStateTransform(prevState, taskState) { + return errors.Errorf("invalid task state transform, from %s to %s", prevState, taskState) + } + for i := 0; i < retryTimes; i++ { - err = d.taskMgr.UpdateGlobalTaskAndAddSubTasks(d.task, newSubTasks) + err = d.taskMgr.UpdateGlobalTaskAndAddSubTasks(d.task, newSubTasks, prevState) if err == nil { break } @@ -306,7 +310,6 @@ func (d *dispatcher) dispatchSubTask(task *proto.Task, handle TaskFlowHandle, me // TODO: Consider using TS. nowTime := time.Now().UTC() task.StartTime = nowTime - task.State = proto.TaskStateRunning task.StateUpdateTime = nowTime retryTimes = nonRetrySQLTime } @@ -341,7 +344,7 @@ func (d *dispatcher) dispatchSubTask(task *proto.Task, handle TaskFlowHandle, me subTasks = append(subTasks, proto.NewSubtask(task.ID, task.Type, instanceID, meta)) } - return d.updateTask(task.State, subTasks, retrySQLTimes) + return d.updateTask(proto.TaskStateRunning, subTasks, retrySQLTimes) } // GenerateSchedulerNodes generate a eligible TiDB nodes. @@ -407,3 +410,59 @@ func (d *dispatcher) WithNewSession(fn func(se sessionctx.Context) error) error func (d *dispatcher) WithNewTxn(ctx context.Context, fn func(se sessionctx.Context) error) error { return d.taskMgr.WithNewTxn(ctx, fn) } + +// VerifyTaskStateTransform verifies whether the task state transform is valid. +func VerifyTaskStateTransform(oldState, newState string) bool { + rules := map[string][]string{ + proto.TaskStatePending: { + proto.TaskStateRunning, + proto.TaskStateCancelling, + proto.TaskStatePausing, + proto.TaskStateSucceed, + proto.TaskStateReverted, + }, + proto.TaskStateRunning: { + proto.TaskStateSucceed, + proto.TaskStateReverting, + proto.TaskStateReverted, + proto.TaskStateCancelling, + proto.TaskStatePausing, + }, + proto.TaskStateSucceed: {}, + proto.TaskStateReverting: { + proto.TaskStateReverted, + // no revert_failed now + // proto.TaskStateRevertFailed, + }, + proto.TaskStateFailed: {}, + proto.TaskStateRevertFailed: {}, + proto.TaskStateCancelling: { + proto.TaskStateReverting, + // no canceled now + // proto.TaskStateCanceled, + }, + proto.TaskStateCanceled: {}, + proto.TaskStatePausing: { + proto.TaskStatePaused, + }, + proto.TaskStatePaused: { + proto.TaskStateResuming, + }, + proto.TaskStateResuming: { + proto.TaskStateRunning, + }, + proto.TaskStateRevertPending: {}, + proto.TaskStateReverted: {}, + } + + if oldState == newState { + return true + } + + for _, state := range rules[oldState] { + if state == newState { + return true + } + } + return false +} diff --git a/disttask/framework/dispatcher/dispatcher_test.go b/disttask/framework/dispatcher/dispatcher_test.go index 57698826e4e1c..2ab371f2b5e12 100644 --- a/disttask/framework/dispatcher/dispatcher_test.go +++ b/disttask/framework/dispatcher/dispatcher_test.go @@ -355,3 +355,25 @@ func (NumberExampleHandle) GetEligibleInstances(ctx context.Context, _ *proto.Ta func (NumberExampleHandle) IsRetryableErr(error) bool { return true } + +func TestVerifyTaskStateTransform(t *testing.T) { + testCases := []struct { + oldState string + newState string + expect bool + }{ + {proto.TaskStateRunning, proto.TaskStateRunning, true}, + {proto.TaskStatePending, proto.TaskStateRunning, true}, + {proto.TaskStatePending, proto.TaskStateReverting, false}, + {proto.TaskStateRunning, proto.TaskStateReverting, true}, + {proto.TaskStateReverting, proto.TaskStateReverted, true}, + {proto.TaskStateReverting, proto.TaskStateSucceed, false}, + {proto.TaskStateRunning, proto.TaskStatePausing, true}, + {proto.TaskStateRunning, proto.TaskStateResuming, false}, + {proto.TaskStateCancelling, proto.TaskStateRunning, false}, + {proto.TaskStateCanceled, proto.TaskStateRunning, false}, + } + for _, tc := range testCases { + require.Equal(t, tc.expect, dispatcher.VerifyTaskStateTransform(tc.oldState, tc.newState)) + } +} diff --git a/disttask/framework/proto/task.go b/disttask/framework/proto/task.go index c900af8da519e..a05eba5592a88 100644 --- a/disttask/framework/proto/task.go +++ b/disttask/framework/proto/task.go @@ -41,6 +41,7 @@ const ( TaskStateCanceled = "canceled" TaskStatePausing = "pausing" TaskStatePaused = "paused" + TaskStateResuming = "resuming" TaskStateRevertPending = "revert_pending" TaskStateReverted = "reverted" ) diff --git a/disttask/framework/storage/table_test.go b/disttask/framework/storage/table_test.go index 18b24d28bcfb2..d25b379999753 100644 --- a/disttask/framework/storage/table_test.go +++ b/disttask/framework/storage/table_test.go @@ -80,8 +80,9 @@ func TestGlobalTaskTable(t *testing.T) { require.Len(t, task4, 1) require.Equal(t, task, task4[0]) + prevState := task.State task.State = proto.TaskStateRunning - err = gm.UpdateGlobalTaskAndAddSubTasks(task, nil) + err = gm.UpdateGlobalTaskAndAddSubTasks(task, nil, prevState) require.NoError(t, err) task5, err := gm.GetGlobalTasksInStates(proto.TaskStateRunning) @@ -238,6 +239,7 @@ func TestBothGlobalAndSubTaskTable(t *testing.T) { require.Equal(t, proto.TaskStatePending, task.State) // isSubTaskRevert: false + prevState := task.State task.State = proto.TaskStateRunning subTasks := []*proto.Subtask{ { @@ -251,7 +253,7 @@ func TestBothGlobalAndSubTaskTable(t *testing.T) { Meta: []byte("m2"), }, } - err = sm.UpdateGlobalTaskAndAddSubTasks(task, subTasks) + err = sm.UpdateGlobalTaskAndAddSubTasks(task, subTasks, prevState) require.NoError(t, err) task, err = sm.GetGlobalTaskByID(1) @@ -275,6 +277,7 @@ func TestBothGlobalAndSubTaskTable(t *testing.T) { require.Equal(t, int64(2), cnt) // isSubTaskRevert: true + prevState = task.State task.State = proto.TaskStateReverting subTasks = []*proto.Subtask{ { @@ -288,7 +291,7 @@ func TestBothGlobalAndSubTaskTable(t *testing.T) { Meta: []byte("m4"), }, } - err = sm.UpdateGlobalTaskAndAddSubTasks(task, subTasks) + err = sm.UpdateGlobalTaskAndAddSubTasks(task, subTasks, prevState) require.NoError(t, err) task, err = sm.GetGlobalTaskByID(1) @@ -317,8 +320,9 @@ func TestBothGlobalAndSubTaskTable(t *testing.T) { defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/disttask/framework/storage/MockUpdateTaskErr")) }() + prevState = task.State task.State = proto.TaskStateFailed - err = sm.UpdateGlobalTaskAndAddSubTasks(task, subTasks) + err = sm.UpdateGlobalTaskAndAddSubTasks(task, subTasks, prevState) require.EqualError(t, err, "updateTaskErr") task, err = sm.GetGlobalTaskByID(1) diff --git a/disttask/framework/storage/task_table.go b/disttask/framework/storage/task_table.go index c0457495706be..aa46c33e356eb 100644 --- a/disttask/framework/storage/task_table.go +++ b/disttask/framework/storage/task_table.go @@ -468,10 +468,10 @@ func (stm *TaskManager) GetSchedulerIDsByTaskID(taskID int64) ([]string, error) } // UpdateGlobalTaskAndAddSubTasks update the global task and add new subtasks -func (stm *TaskManager) UpdateGlobalTaskAndAddSubTasks(gTask *proto.Task, subtasks []*proto.Subtask) error { +func (stm *TaskManager) UpdateGlobalTaskAndAddSubTasks(gTask *proto.Task, subtasks []*proto.Subtask, prevState string) error { return stm.WithNewTxn(stm.ctx, func(se sessionctx.Context) error { - _, err := ExecSQL(stm.ctx, se, "update mysql.tidb_global_task set state = %?, dispatcher_id = %?, step = %?, state_update_time = %?, concurrency = %?, meta = %?, error = %? where id = %?", - gTask.State, gTask.DispatcherID, gTask.Step, gTask.StateUpdateTime.UTC().String(), gTask.Concurrency, gTask.Meta, serializeErr(gTask.Error), gTask.ID) + _, err := ExecSQL(stm.ctx, se, "update mysql.tidb_global_task set state = %?, dispatcher_id = %?, step = %?, state_update_time = %?, concurrency = %?, meta = %?, error = %? where id = %? and state = %?", + gTask.State, gTask.DispatcherID, gTask.Step, gTask.StateUpdateTime.UTC().String(), gTask.Concurrency, gTask.Meta, serializeErr(gTask.Error), gTask.ID, prevState) if err != nil { return err } From ddb7b36cb6e37206d4e7f810c2b4723fd58a4500 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Fri, 18 Aug 2023 15:36:39 +0800 Subject: [PATCH 11/32] *: Add system variable `tidb_session_alias` to log a custom field `session_alias` in session log (#46072) close pingcap/tidb#46071 --- parser/model/model.go | 8 ++++ planner/core/preprocess.go | 52 +++++++++------------- server/BUILD.bazel | 1 + server/conn.go | 18 +++++++- session/test/variable/BUILD.bazel | 2 +- session/test/variable/variable_test.go | 21 +++++++++ sessionctx/variable/session.go | 3 ++ sessionctx/variable/sysvar.go | 20 +++++++++ sessionctx/variable/tidb_vars.go | 2 + util/logutil/BUILD.bazel | 3 ++ util/logutil/log.go | 60 ++++++++++++++++++-------- util/logutil/log_test.go | 40 ++++++++++++++++- util/logutil/main_test.go | 2 + util/tracing/BUILD.bazel | 2 + util/tracing/util.go | 22 ++++++++++ util/tracing/util_test.go | 20 +++++++++ util/util.go | 12 ++++++ util/util_test.go | 18 ++++++++ 18 files changed, 251 insertions(+), 55 deletions(-) diff --git a/parser/model/model.go b/parser/model/model.go index 64cda27eb4d53..1402afabd7983 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -2167,3 +2167,11 @@ func (s WindowRepeatType) String() string { return "" } } + +// TraceInfo is the information for trace. +type TraceInfo struct { + // ConnectionID is the id of the connection + ConnectionID uint64 `json:"connection_id"` + // SessionAlias is the alias of session + SessionAlias string `json:"session_alias"` +} diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 7b4cdb58c1aba..b231939ce3081 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -785,32 +785,32 @@ func (p *preprocessor) checkSetOprSelectList(stmt *ast.SetOprSelectList) { } func (p *preprocessor) checkCreateDatabaseGrammar(stmt *ast.CreateDatabaseStmt) { - if isIncorrectName(stmt.Name.L) { + if util.IsInCorrectIdentifierName(stmt.Name.L) { p.err = dbterror.ErrWrongDBName.GenWithStackByArgs(stmt.Name) } } func (p *preprocessor) checkAlterDatabaseGrammar(stmt *ast.AlterDatabaseStmt) { // for 'ALTER DATABASE' statement, database name can be empty to alter default database. - if isIncorrectName(stmt.Name.L) && !stmt.AlterDefaultDatabase { + if util.IsInCorrectIdentifierName(stmt.Name.L) && !stmt.AlterDefaultDatabase { p.err = dbterror.ErrWrongDBName.GenWithStackByArgs(stmt.Name) } } func (p *preprocessor) checkDropDatabaseGrammar(stmt *ast.DropDatabaseStmt) { - if isIncorrectName(stmt.Name.L) { + if util.IsInCorrectIdentifierName(stmt.Name.L) { p.err = dbterror.ErrWrongDBName.GenWithStackByArgs(stmt.Name) } } func (p *preprocessor) checkFlashbackTableGrammar(stmt *ast.FlashBackTableStmt) { - if isIncorrectName(stmt.NewName) { + if util.IsInCorrectIdentifierName(stmt.NewName) { p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(stmt.NewName) } } func (p *preprocessor) checkFlashbackDatabaseGrammar(stmt *ast.FlashBackDatabaseStmt) { - if isIncorrectName(stmt.NewName) { + if util.IsInCorrectIdentifierName(stmt.NewName) { p.err = dbterror.ErrWrongDBName.GenWithStackByArgs(stmt.NewName) } } @@ -874,7 +874,7 @@ func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) { } } tName := stmt.Table.Name.String() - if isIncorrectName(tName) { + if util.IsInCorrectIdentifierName(tName) { p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(tName) return } @@ -938,7 +938,7 @@ func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) { if stmt.Partition != nil { for _, def := range stmt.Partition.Definitions { pName := def.Name.String() - if isIncorrectName(pName) { + if util.IsInCorrectIdentifierName(pName) { p.err = dbterror.ErrWrongPartitionName.GenWithStackByArgs() return } @@ -948,12 +948,12 @@ func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) { func (p *preprocessor) checkCreateViewGrammar(stmt *ast.CreateViewStmt) { vName := stmt.ViewName.Name.String() - if isIncorrectName(vName) { + if util.IsInCorrectIdentifierName(vName) { p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(vName) return } for _, col := range stmt.Cols { - if isIncorrectName(col.String()) { + if util.IsInCorrectIdentifierName(col.String()) { p.err = dbterror.ErrWrongColumnName.GenWithStackByArgs(col) return } @@ -1014,7 +1014,7 @@ func (p *preprocessor) checkDropTableGrammar(stmt *ast.DropTableStmt) { func (p *preprocessor) checkDropTemporaryTableGrammar(stmt *ast.DropTableStmt) { currentDB := model.NewCIStr(p.sctx.GetSessionVars().CurrentDB) for _, t := range stmt.Tables { - if isIncorrectName(t.Name.String()) { + if util.IsInCorrectIdentifierName(t.Name.String()) { p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(t.Name.String()) return } @@ -1045,7 +1045,7 @@ func (p *preprocessor) checkDropTemporaryTableGrammar(stmt *ast.DropTableStmt) { func (p *preprocessor) checkDropTableNames(tables []*ast.TableName) { for _, t := range tables { - if isIncorrectName(t.Name.String()) { + if util.IsInCorrectIdentifierName(t.Name.String()) { p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(t.Name.String()) return } @@ -1118,7 +1118,7 @@ func checkColumnOptions(isTempTable bool, ops []*ast.ColumnOption) (int, error) func (p *preprocessor) checkCreateIndexGrammar(stmt *ast.CreateIndexStmt) { tName := stmt.Table.Name.String() - if isIncorrectName(tName) { + if util.IsInCorrectIdentifierName(tName) { p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(tName) return } @@ -1152,12 +1152,12 @@ func (p *preprocessor) checkRenameTableGrammar(stmt *ast.RenameTableStmt) { } func (p *preprocessor) checkRenameTable(oldTable, newTable string) { - if isIncorrectName(oldTable) { + if util.IsInCorrectIdentifierName(oldTable) { p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(oldTable) return } - if isIncorrectName(newTable) { + if util.IsInCorrectIdentifierName(newTable) { p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(newTable) return } @@ -1182,7 +1182,7 @@ func (p *preprocessor) checkRepairTableGrammar(stmt *ast.RepairTableStmt) { func (p *preprocessor) checkAlterTableGrammar(stmt *ast.AlterTableStmt) { tName := stmt.Table.Name.String() - if isIncorrectName(tName) { + if util.IsInCorrectIdentifierName(tName) { p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(tName) return } @@ -1190,7 +1190,7 @@ func (p *preprocessor) checkAlterTableGrammar(stmt *ast.AlterTableStmt) { for _, spec := range specs { if spec.NewTable != nil { ntName := spec.NewTable.Name.String() - if isIncorrectName(ntName) { + if util.IsInCorrectIdentifierName(ntName) { p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(ntName) return } @@ -1217,7 +1217,7 @@ func (p *preprocessor) checkAlterTableGrammar(stmt *ast.AlterTableStmt) { } case ast.AlterTableAddStatistics, ast.AlterTableDropStatistics: statsName := spec.Statistics.StatsName - if isIncorrectName(statsName) { + if util.IsInCorrectIdentifierName(statsName) { msg := fmt.Sprintf("Incorrect statistics name: %s", statsName) p.err = ErrInternal.GenWithStack(msg) return @@ -1225,7 +1225,7 @@ func (p *preprocessor) checkAlterTableGrammar(stmt *ast.AlterTableStmt) { case ast.AlterTableAddPartitions: for _, def := range spec.PartDefinitions { pName := def.Name.String() - if isIncorrectName(pName) { + if util.IsInCorrectIdentifierName(pName) { p.err = dbterror.ErrWrongPartitionName.GenWithStackByArgs() return } @@ -1334,7 +1334,7 @@ func checkReferInfoForTemporaryTable(tableMetaInfo *model.TableInfo) error { func checkColumn(colDef *ast.ColumnDef) error { // Check column name. cName := colDef.Name.Name.String() - if isIncorrectName(cName) { + if util.IsInCorrectIdentifierName(cName) { return dbterror.ErrWrongColumnName.GenWithStackByArgs(cName) } @@ -1457,18 +1457,6 @@ func isInvalidDefaultValue(colDef *ast.ColumnDef) bool { return false } -// isIncorrectName checks if the identifier is incorrect. -// See https://dev.mysql.com/doc/refman/5.7/en/identifiers.html -func isIncorrectName(name string) bool { - if len(name) == 0 { - return true - } - if name[len(name)-1] == ' ' { - return true - } - return false -} - // checkContainDotColumn checks field contains the table name. // for example :create table t (c1.c2 int default null). func (p *preprocessor) checkContainDotColumn(stmt *ast.CreateTableStmt) { @@ -1683,7 +1671,7 @@ func (p *preprocessor) resolveAlterTableStmt(node *ast.AlterTableStmt) { func (p *preprocessor) resolveCreateSequenceStmt(stmt *ast.CreateSequenceStmt) { sName := stmt.Name.Name.String() - if isIncorrectName(sName) { + if util.IsInCorrectIdentifierName(sName) { p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(sName) return } diff --git a/server/BUILD.bazel b/server/BUILD.bazel index 590485e673fe1..1b8e22be32874 100644 --- a/server/BUILD.bazel +++ b/server/BUILD.bazel @@ -36,6 +36,7 @@ go_library( "//parser/ast", "//parser/auth", "//parser/charset", + "//parser/model", "//parser/mysql", "//parser/terror", "//planner/core", diff --git a/server/conn.go b/server/conn.go index 94d39eda35656..0d5ce53e3b137 100644 --- a/server/conn.go +++ b/server/conn.go @@ -67,6 +67,7 @@ import ( "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/auth" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" plannercore "github.com/pingcap/tidb/planner/core" @@ -977,17 +978,30 @@ func (cc *clientConn) Run(ctx context.Context) { close(cc.quit) }() + parentCtx := ctx + var traceInfo *model.TraceInfo // Usually, client connection status changes between [dispatching] <=> [reading]. // When some event happens, server may notify this client connection by setting // the status to special values, for example: kill or graceful shutdown. // The client connection would detect the events when it fails to change status // by CAS operation, it would then take some actions accordingly. for { + sessVars := cc.ctx.GetSessionVars() + if alias := sessVars.SessionAlias; traceInfo == nil || traceInfo.SessionAlias != alias { + // We should reset the context trace info when traceInfo not inited or session alias changed. + traceInfo = &model.TraceInfo{ + ConnectionID: cc.connectionID, + SessionAlias: alias, + } + ctx = logutil.WithSessionAlias(parentCtx, sessVars.SessionAlias) + ctx = tracing.ContextWithTraceInfo(ctx, traceInfo) + } + // Close connection between txn when we are going to shutdown server. // Note the current implementation when shutting down, for an idle connection, the connection may block at readPacket() // consider provider a way to close the connection directly after sometime if we can not read any data. if cc.server.inShutdownMode.Load() { - if !cc.ctx.GetSessionVars().InTxn() { + if !sessVars.InTxn() { return } } @@ -1216,7 +1230,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { defer task.End() trace.Log(ctx, "sql", lc.String()) - ctx = logutil.WithTraceLogger(ctx, cc.connectionID) + ctx = logutil.WithTraceLogger(ctx, tracing.TraceInfoFromContext(ctx)) taskID := *(*uint64)(unsafe.Pointer(task)) ctx = pprof.WithLabels(ctx, pprof.Labels("trace", strconv.FormatUint(taskID, 10))) diff --git a/session/test/variable/BUILD.bazel b/session/test/variable/BUILD.bazel index 9c0c49db0ceec..2edfbf857341e 100644 --- a/session/test/variable/BUILD.bazel +++ b/session/test/variable/BUILD.bazel @@ -8,7 +8,7 @@ go_test( "variable_test.go", ], flaky = True, - shard_count = 21, + shard_count = 22, deps = [ "//config", "//kv", diff --git a/session/test/variable/variable_test.go b/session/test/variable/variable_test.go index 6c9624f629d95..00a5089a28704 100644 --- a/session/test/variable/variable_test.go +++ b/session/test/variable/variable_test.go @@ -608,3 +608,24 @@ func TestSysdateIsNow(t *testing.T) { tk.MustQuery("show variables like '%tidb_sysdate_is_now%'").Check(testkit.Rows("tidb_sysdate_is_now ON")) require.True(t, tk.Session().GetSessionVars().SysdateIsNow) } + +func TestSessionAlias(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustQuery("select @@tidb_session_alias").Check(testkit.Rows("")) + // normal set + tk.MustExec("set @@tidb_session_alias='alias123'") + tk.MustQuery("select @@tidb_session_alias").Check(testkit.Rows("alias123")) + // set a long value + val := "0123456789012345678901234567890123456789012345678901234567890123456789" + tk.MustExec("set @@tidb_session_alias=?", val) + tk.MustQuery("select @@tidb_session_alias").Check(testkit.Rows(val[:64])) + // an invalid value + err := tk.ExecToErr("set @@tidb_session_alias='abc '") + require.EqualError(t, err, "[variable:1231]Incorrect value for variable @@tidb_session_alias 'abc '") + tk.MustQuery("select @@tidb_session_alias").Check(testkit.Rows(val[:64])) + // reset to empty + tk.MustExec("set @@tidb_session_alias=''") + tk.MustQuery("select @@tidb_session_alias").Check(testkit.Rows("")) +} diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index f9eb3714dee67..99018080e4c75 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1534,6 +1534,9 @@ type SessionVars struct { // When set to true, skip missing partition stats and continue to merge other partition stats to global stats. // When set to false, give up merging partition stats to global stats. SkipMissingPartitionStats bool + + // SessionAlias is the identifier of the session + SessionAlias string } // GetOptimizerFixControlMap returns the specified value of the optimizer fix control. diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index bcf1f4efe8337..eeb51efbe9a57 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" _ "github.com/pingcap/tidb/types/parser_driver" // for parser driver + "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/gctuner" "github.com/pingcap/tidb/util/logutil" @@ -2792,6 +2793,25 @@ var defaultSysVars = []*SysVar{ }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return BoolToOnOff(EnableCheckConstraint.Load()), nil }}, + {Scope: ScopeSession, Name: TiDBSessionAlias, Value: "", Type: TypeStr, + Validation: func(s *SessionVars, normalizedValue string, originalValue string, _ ScopeFlag) (string, error) { + if len(normalizedValue) > 64 { + s.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(TiDBSessionAlias, originalValue)) + normalizedValue = normalizedValue[:64] + } + + if len(normalizedValue) > 0 && util.IsInCorrectIdentifierName(normalizedValue) { + return "", ErrWrongValueForVar.GenWithStack("Incorrect value for variable @@%s '%s'", TiDBSessionAlias, normalizedValue) + } + + return normalizedValue, nil + }, + SetSession: func(vars *SessionVars, s string) error { + vars.SessionAlias = s + return nil + }, GetSession: func(vars *SessionVars) (string, error) { + return vars.SessionAlias, nil + }}, } func setTiFlashComputeDispatchPolicy(s *SessionVars, val string) error { diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 89159050c2f5e..381c86c39daa6 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -1083,6 +1083,8 @@ const ( // When set to true, skip missing partition stats and continue to merge other partition stats to global stats. // When set to false, give up merging partition stats to global stats. TiDBSkipMissingPartitionStats = "tidb_skip_missing_partition_stats" + // TiDBSessionAlias indicates the alias of a session which is used for tracing. + TiDBSessionAlias = "tidb_session_alias" ) // TiDB intentional limits diff --git a/util/logutil/BUILD.bazel b/util/logutil/BUILD.bazel index 5b3bf9d7a8da1..c032019d3c608 100644 --- a/util/logutil/BUILD.bazel +++ b/util/logutil/BUILD.bazel @@ -10,6 +10,7 @@ go_library( importpath = "github.com/pingcap/tidb/util/logutil", visibility = ["//visibility:public"], deps = [ + "//parser/model", "@com_github_golang_protobuf//proto", "@com_github_grpc_ecosystem_go_grpc_middleware//logging/zap", "@com_github_opentracing_opentracing_go//:opentracing-go", @@ -35,7 +36,9 @@ go_test( flaky = True, deps = [ "//kv", + "//parser/model", "//testkit/testsetup", + "@com_github_google_uuid//:uuid", "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pingcap_log//:log", "@com_github_stretchr_testify//require", diff --git a/util/logutil/log.go b/util/logutil/log.go index 0e48e5ce64678..b01ea86987fd9 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -27,6 +27,7 @@ import ( tlog "github.com/opentracing/opentracing-go/log" "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/pingcap/tidb/parser/model" "github.com/tikv/client-go/v2/tikv" "go.uber.org/zap" "go.uber.org/zap/zapcore" @@ -201,43 +202,56 @@ func BgLogger() *zap.Logger { // WithConnID attaches connId to context. func WithConnID(ctx context.Context, connID uint64) context.Context { - var logger *zap.Logger - if ctxLogger, ok := ctx.Value(CtxLogKey).(*zap.Logger); ok { - logger = ctxLogger - } else { - logger = log.L() - } - return context.WithValue(ctx, CtxLogKey, logger.With(zap.Uint64("conn", connID))) + return WithFields(ctx, zap.Uint64("conn", connID)) +} + +// WithSessionAlias attaches session_alias to context +func WithSessionAlias(ctx context.Context, alias string) context.Context { + return WithFields(ctx, zap.String("session_alias", alias)) } // WithCategory attaches category to context. func WithCategory(ctx context.Context, category string) context.Context { - var logger *zap.Logger - if ctxLogger, ok := ctx.Value(CtxLogKey).(*zap.Logger); ok { - logger = ctxLogger - } else { - logger = log.L() + return WithFields(ctx, zap.String("category", category)) +} + +func fieldsFromTraceInfo(info *model.TraceInfo) []zap.Field { + if info == nil { + return nil + } + + fields := make([]zap.Field, 0, 2) + if info.ConnectionID != 0 { + fields = append(fields, zap.Uint64("conn", info.ConnectionID)) } - return context.WithValue(ctx, CtxLogKey, logger.With(zap.String("category", category))) + + if info.SessionAlias != "" { + fields = append(fields, zap.String("session_alias", info.SessionAlias)) + } + + return fields } // WithTraceLogger attaches trace identifier to context -func WithTraceLogger(ctx context.Context, connID uint64) context.Context { +func WithTraceLogger(ctx context.Context, info *model.TraceInfo) context.Context { var logger *zap.Logger if ctxLogger, ok := ctx.Value(CtxLogKey).(*zap.Logger); ok { logger = ctxLogger } else { logger = log.L() } - return context.WithValue(ctx, CtxLogKey, wrapTraceLogger(ctx, connID, logger)) + return context.WithValue(ctx, CtxLogKey, wrapTraceLogger(ctx, info, logger)) } -func wrapTraceLogger(ctx context.Context, connID uint64, logger *zap.Logger) *zap.Logger { +func wrapTraceLogger(ctx context.Context, info *model.TraceInfo, logger *zap.Logger) *zap.Logger { return logger.WithOptions(zap.WrapCore(func(core zapcore.Core) zapcore.Core { tl := &traceLog{ctx: ctx} // cfg.Format == "", never return error enc, _ := log.NewTextEncoder(&log.Config{}) - traceCore := log.NewTextCore(enc, tl, tl).With([]zapcore.Field{zap.Uint64("conn", connID)}) + traceCore := log.NewTextCore(enc, tl, tl) + if fields := fieldsFromTraceInfo(info); len(fields) > 0 { + traceCore = traceCore.With(fields) + } return zapcore.NewTee(traceCore, core) })) } @@ -261,13 +275,23 @@ func (*traceLog) Sync() error { // WithKeyValue attaches key/value to context. func WithKeyValue(ctx context.Context, key, value string) context.Context { + return WithFields(ctx, zap.String(key, value)) +} + +// WithFields attaches key/value to context. +func WithFields(ctx context.Context, fields ...zap.Field) context.Context { var logger *zap.Logger if ctxLogger, ok := ctx.Value(CtxLogKey).(*zap.Logger); ok { logger = ctxLogger } else { logger = log.L() } - return context.WithValue(ctx, CtxLogKey, logger.With(zap.String(key, value))) + + if len(fields) > 0 { + logger = logger.With(fields...) + } + + return context.WithValue(ctx, CtxLogKey, logger) } // TraceEventKey presents the TraceEventKey in span log. diff --git a/util/logutil/log_test.go b/util/logutil/log_test.go index 1985ae43ed0e5..975c80d16200d 100644 --- a/util/logutil/log_test.go +++ b/util/logutil/log_test.go @@ -17,17 +17,37 @@ package logutil import ( "bufio" "context" + "fmt" "io" "os" "runtime" "testing" + "github.com/google/uuid" "github.com/pingcap/log" + "github.com/pingcap/tidb/parser/model" "github.com/stretchr/testify/require" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) +func TestFieldsFromTraceInfo(t *testing.T) { + fields := fieldsFromTraceInfo(nil) + require.Equal(t, 0, len(fields)) + + fields = fieldsFromTraceInfo(&model.TraceInfo{}) + require.Equal(t, 0, len(fields)) + + fields = fieldsFromTraceInfo(&model.TraceInfo{ConnectionID: 1}) + require.Equal(t, []zap.Field{zap.Uint64("conn", 1)}, fields) + + fields = fieldsFromTraceInfo(&model.TraceInfo{SessionAlias: "alias123"}) + require.Equal(t, []zap.Field{zap.String("session_alias", "alias123")}, fields) + + fields = fieldsFromTraceInfo(&model.TraceInfo{ConnectionID: 1, SessionAlias: "alias123"}) + require.Equal(t, []zap.Field{zap.Uint64("conn", 1), zap.String("session_alias", "alias123")}, fields) +} + func TestZapLoggerWithKeys(t *testing.T) { if runtime.GOOS == "windows" { // Skip this test on windows for two reason: @@ -36,7 +56,7 @@ func TestZapLoggerWithKeys(t *testing.T) { t.Skip("skip on windows") } - fileCfg := FileLogConfig{log.FileLogConfig{Filename: "zap_log", MaxSize: 4096}} + fileCfg := FileLogConfig{log.FileLogConfig{Filename: fmt.Sprintf("zap_log_%s", uuid.NewString()), MaxSize: 4096}} conf := NewLogConfig("info", DefaultLogFormat, "", fileCfg, false) err := InitLogger(conf) require.NoError(t, err) @@ -46,11 +66,27 @@ func TestZapLoggerWithKeys(t *testing.T) { err = os.Remove(fileCfg.Filename) require.NoError(t, err) + conf = NewLogConfig("info", DefaultLogFormat, "", fileCfg, false) + err = InitLogger(conf) + require.NoError(t, err) + ctx = WithConnID(context.Background(), connID) + ctx = WithSessionAlias(ctx, "alias123") + testZapLogger(ctx, t, fileCfg.Filename, zapLogWithTraceInfoPattern) + err = os.Remove(fileCfg.Filename) + require.NoError(t, err) + + err = InitLogger(conf) + require.NoError(t, err) + ctx1 := WithFields(context.Background(), zap.Int64("conn", 123), zap.String("session_alias", "alias456")) + testZapLogger(ctx1, t, fileCfg.Filename, zapLogWithTraceInfoPattern) + err = os.Remove(fileCfg.Filename) + require.NoError(t, err) + err = InitLogger(conf) require.NoError(t, err) key := "ctxKey" val := "ctxValue" - ctx1 := WithKeyValue(context.Background(), key, val) + ctx1 = WithKeyValue(context.Background(), key, val) testZapLogger(ctx1, t, fileCfg.Filename, zapLogWithKeyValPatternByCtx) err = os.Remove(fileCfg.Filename) require.NoError(t, err) diff --git a/util/logutil/main_test.go b/util/logutil/main_test.go index f2a143eee0c23..0f6b850e693a9 100644 --- a/util/logutil/main_test.go +++ b/util/logutil/main_test.go @@ -25,6 +25,8 @@ const ( // zapLogPatern is used to match the zap log format, such as the following log: // [2019/02/13 15:56:05.385 +08:00] [INFO] [log_test.go:167] ["info message"] [conn=conn1] ["str key"=val] ["int key"=123] zapLogWithConnIDPattern = `\[\d\d\d\d/\d\d/\d\d \d\d:\d\d:\d\d.\d\d\d\ (\+|-)\d\d:\d\d\] \[(FATAL|ERROR|WARN|INFO|DEBUG)\] \[([\w_%!$@.,+~-]+|\\.)+:\d+\] \[.*\] \[conn=.*\] (\[.*=.*\]).*\n` + // [2019/02/13 15:56:05.385 +08:00] [INFO] [log_test.go:167] ["info message"] [conn=conn1] [session_alias=alias] ["str key"=val] ["int key"=123] + zapLogWithTraceInfoPattern = `\[\d\d\d\d/\d\d/\d\d \d\d:\d\d:\d\d.\d\d\d\ (\+|-)\d\d:\d\d\] \[(FATAL|ERROR|WARN|INFO|DEBUG)\] \[([\w_%!$@.,+~-]+|\\.)+:\d+\] \[.*\] \[conn=.*\] \[session_alias=.*\] (\[.*=.*\]).*\n` // [2019/02/13 15:56:05.385 +08:00] [INFO] [log_test.go:167] ["info message"] [ctxKey=ctxKey1] ["str key"=val] ["int key"=123] zapLogWithKeyValPatternByCtx = `\[\d\d\d\d/\d\d/\d\d \d\d:\d\d:\d\d.\d\d\d\ (\+|-)\d\d:\d\d\] \[(FATAL|ERROR|WARN|INFO|DEBUG)\] \[([\w_%!$@.,+~-]+|\\.)+:\d+\] \[.*\] \[ctxKey=.*\] (\[.*=.*\]).*\n` // [2019/02/13 15:56:05.385 +08:00] [INFO] [log_test.go:167] ["info message"] [coreKey=coreKey1] ["str key"=val] ["int key"=123] diff --git a/util/tracing/BUILD.bazel b/util/tracing/BUILD.bazel index 9f6ccac4a590e..ea2cada504ca3 100644 --- a/util/tracing/BUILD.bazel +++ b/util/tracing/BUILD.bazel @@ -9,6 +9,7 @@ go_library( importpath = "github.com/pingcap/tidb/util/tracing", visibility = ["//visibility:public"], deps = [ + "//parser/model", "@com_github_opentracing_basictracer_go//:basictracer-go", "@com_github_opentracing_opentracing_go//:opentracing-go", ], @@ -26,6 +27,7 @@ go_test( embed = [":tracing"], flaky = True, deps = [ + "//parser/model", "//testkit/testsetup", "@com_github_opentracing_basictracer_go//:basictracer-go", "@com_github_opentracing_opentracing_go//:opentracing-go", diff --git a/util/tracing/util.go b/util/tracing/util.go index 924e2eb039f44..e26a15340cd96 100644 --- a/util/tracing/util.go +++ b/util/tracing/util.go @@ -20,11 +20,16 @@ import ( "github.com/opentracing/basictracer-go" "github.com/opentracing/opentracing-go" + "github.com/pingcap/tidb/parser/model" ) // TiDBTrace is set as Baggage on traces which are used for tidb tracing. const TiDBTrace = "tr" +type sqlTracingCtxKeyType struct{} + +var sqlTracingCtxKey = sqlTracingCtxKeyType{} + // A CallbackRecorder immediately invokes itself on received trace spans. type CallbackRecorder func(sp basictracer.RawSpan) @@ -110,3 +115,20 @@ func (r Region) End() { } r.Region.End() } + +// TraceInfoFromContext returns the `model.TraceInfo` in context +func TraceInfoFromContext(ctx context.Context) *model.TraceInfo { + val := ctx.Value(sqlTracingCtxKey) + if info, ok := val.(*model.TraceInfo); ok { + return info + } + return nil +} + +// ContextWithTraceInfo creates a new `model.TraceInfo` for context +func ContextWithTraceInfo(ctx context.Context, info *model.TraceInfo) context.Context { + if info == nil { + return ctx + } + return context.WithValue(ctx, sqlTracingCtxKey, info) +} diff --git a/util/tracing/util_test.go b/util/tracing/util_test.go index 119f2777017b9..3af7ac5b99fa4 100644 --- a/util/tracing/util_test.go +++ b/util/tracing/util_test.go @@ -20,6 +20,7 @@ import ( "github.com/opentracing/basictracer-go" "github.com/opentracing/opentracing-go" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/tracing" "github.com/stretchr/testify/require" ) @@ -130,3 +131,22 @@ func TestTreeRelationship(t *testing.T) { require.Equal(t, collectedSpans[1].Context.SpanID, collectedSpans[2].ParentSpanID) } } + +func TestTraceInfoFromContext(t *testing.T) { + ctx := context.Background() + // get info from a non-tracing context + require.Nil(t, tracing.TraceInfoFromContext(ctx)) + // ContextWithTraceInfo with a nil info will return the original context + require.Equal(t, ctx, tracing.ContextWithTraceInfo(ctx, nil)) + // create a context with trace info + ctx, cancel := context.WithCancel(context.WithValue(ctx, "val1", "a")) + ctx = tracing.ContextWithTraceInfo(ctx, &model.TraceInfo{ConnectionID: 12345, SessionAlias: "alias1"}) + // new context should have the same value as the original one + info := tracing.TraceInfoFromContext(ctx) + require.Equal(t, uint64(12345), info.ConnectionID) + require.Equal(t, "alias1", info.SessionAlias) + require.Equal(t, "a", ctx.Value("val1")) + require.NoError(t, ctx.Err()) + cancel() + require.Error(t, ctx.Err()) +} diff --git a/util/util.go b/util/util.go index 271fa32730784..69882cf3aed19 100644 --- a/util/util.go +++ b/util/util.go @@ -283,3 +283,15 @@ func ReadLines(reader *bufio.Reader, count int, maxLineSize int) ([][]byte, erro } return lines, nil } + +// IsInCorrectIdentifierName checks if the identifier is incorrect. +// See https://dev.mysql.com/doc/refman/5.7/en/identifiers.html +func IsInCorrectIdentifierName(name string) bool { + if len(name) == 0 { + return true + } + if name[len(name)-1] == ' ' { + return true + } + return false +} diff --git a/util/util_test.go b/util/util_test.go index 18f5aa8173060..71cbe8e18f84d 100644 --- a/util/util_test.go +++ b/util/util_test.go @@ -94,3 +94,21 @@ line3`)) require.Equal(t, io.EOF, err) require.Len(t, line, 0) } + +func TestIsInCorrectIdentifierName(t *testing.T) { + tests := []struct { + name string + input string + correct bool + }{ + {"Empty identifier", "", true}, + {"Ending space", "test ", true}, + {"Correct identifier", "test", false}, + {"Other correct Identifier", "aaa --\n\txyz", false}, + } + + for _, tc := range tests { + got := IsInCorrectIdentifierName(tc.input) + require.Equalf(t, tc.correct, got, "IsInCorrectIdentifierName(%v) != %v", tc.name, tc.correct) + } +} From b1f088040ea1983d14c69f2e481ee5eb97121ebe Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Fri, 18 Aug 2023 15:36:46 +0800 Subject: [PATCH 12/32] *: simple code (#46218) --- infoschema/builder.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/infoschema/builder.go b/infoschema/builder.go index 4b6c6627d100b..6008a30d21f65 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -834,12 +834,10 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i tableNames := b.is.schemaMap[dbInfo.Name.L] tableNames.tables[tblInfo.Name.L] = tbl bucketIdx := tableBucketIdx(tableID) - sortedTbls := b.is.sortedTablesBuckets[bucketIdx] - sortedTbls = append(sortedTbls, tbl) - slices.SortFunc(sortedTbls, func(i, j table.Table) int { + b.is.sortedTablesBuckets[bucketIdx] = append(b.is.sortedTablesBuckets[bucketIdx], tbl) + slices.SortFunc(b.is.sortedTablesBuckets[bucketIdx], func(i, j table.Table) int { return cmp.Compare(i.Meta().ID, j.Meta().ID) }) - b.is.sortedTablesBuckets[bucketIdx] = sortedTbls if tblInfo.TempTableType != model.TempTableNone { b.addTemporaryTable(tableID) From dcd4667ae98127fb4421571183c7ad3e107dd5b0 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Fri, 18 Aug 2023 17:12:32 +0800 Subject: [PATCH 13/32] *: use std/slices to replace exp/slices (#46221) ref pingcap/tidb#45933 --- br/pkg/lightning/backend/local/duplicate.go | 2 +- br/pkg/lightning/backend/local/localhelper.go | 14 +++++--------- br/pkg/lightning/importer/BUILD.bazel | 1 - br/pkg/lightning/importer/table_import.go | 5 +++-- br/pkg/lightning/mydump/BUILD.bazel | 1 - br/pkg/lightning/mydump/csv_parser.go | 2 +- cmd/mirror/BUILD.bazel | 1 - cmd/mirror/mirror.go | 2 +- ddl/job_table.go | 2 +- executor/aggregate.go | 5 +++-- executor/infoschema_reader.go | 6 ++---- executor/memtable_reader.go | 7 ++++--- executor/show_stats.go | 7 ++++--- parser/model/model.go | 5 +++-- store/copr/BUILD.bazel | 1 - store/copr/batch_coprocessor.go | 17 +++++++++-------- store/helper/BUILD.bazel | 1 - store/helper/helper.go | 11 ++++++----- store/mockstore/unistore/tikv/BUILD.bazel | 1 - store/mockstore/unistore/tikv/mock_region.go | 6 ++---- timer/api/store.go | 2 +- 21 files changed, 46 insertions(+), 53 deletions(-) diff --git a/br/pkg/lightning/backend/local/duplicate.go b/br/pkg/lightning/backend/local/duplicate.go index 8952dfb3c7f21..69c1a1feec8c0 100644 --- a/br/pkg/lightning/backend/local/duplicate.go +++ b/br/pkg/lightning/backend/local/duplicate.go @@ -20,6 +20,7 @@ import ( "fmt" "io" "math" + "slices" "sync" "github.com/cockroachdb/pebble" @@ -53,7 +54,6 @@ import ( kvutil "github.com/tikv/client-go/v2/util" "go.uber.org/atomic" "go.uber.org/zap" - "golang.org/x/exp/slices" "golang.org/x/sync/errgroup" "golang.org/x/time/rate" ) diff --git a/br/pkg/lightning/backend/local/localhelper.go b/br/pkg/lightning/backend/local/localhelper.go index 8bc5ffd21dbdf..060a7b28ebcbf 100644 --- a/br/pkg/lightning/backend/local/localhelper.go +++ b/br/pkg/lightning/backend/local/localhelper.go @@ -19,6 +19,7 @@ import ( "context" "database/sql" "math" + "slices" "sort" "strings" "sync" @@ -43,7 +44,6 @@ import ( "github.com/pingcap/tidb/util/mathutil" "go.uber.org/multierr" "go.uber.org/zap" - "golang.org/x/exp/slices" "golang.org/x/sync/errgroup" "golang.org/x/time/rate" ) @@ -271,9 +271,7 @@ func (local *Backend) SplitAndScatterRegionByRanges( var err1 error region := sp.region keys := sp.keys - slices.SortFunc(keys, func(i, j []byte) bool { - return bytes.Compare(i, j) < 0 - }) + slices.SortFunc(keys, bytes.Compare) splitRegion := region startIdx := 0 endIdx := 0 @@ -317,8 +315,8 @@ func (local *Backend) SplitAndScatterRegionByRanges( log.FromContext(ctx).Info("batch split region", zap.Uint64("region_id", splitRegion.Region.Id), zap.Int("keys", endIdx-startIdx), zap.Binary("firstKey", keys[startIdx]), zap.Binary("end", keys[endIdx-1])) - slices.SortFunc(newRegions, func(i, j *split.RegionInfo) bool { - return bytes.Compare(i.Region.StartKey, j.Region.StartKey) < 0 + slices.SortFunc(newRegions, func(i, j *split.RegionInfo) int { + return bytes.Compare(i.Region.StartKey, j.Region.StartKey) }) syncLock.Lock() scatterRegions = append(scatterRegions, newRegions...) @@ -372,9 +370,7 @@ func (local *Backend) SplitAndScatterRegionByRanges( if len(retryKeys) == 0 { break } - slices.SortFunc(retryKeys, func(i, j []byte) bool { - return bytes.Compare(i, j) < 0 - }) + slices.SortFunc(retryKeys, bytes.Compare) minKey = codec.EncodeBytes([]byte{}, retryKeys[0]) maxKey = codec.EncodeBytes([]byte{}, nextKey(retryKeys[len(retryKeys)-1])) } diff --git a/br/pkg/lightning/importer/BUILD.bazel b/br/pkg/lightning/importer/BUILD.bazel index ecc31bc3c7354..e52caf6f7aeee 100644 --- a/br/pkg/lightning/importer/BUILD.bazel +++ b/br/pkg/lightning/importer/BUILD.bazel @@ -94,7 +94,6 @@ go_library( "@org_golang_google_grpc//codes", "@org_golang_google_grpc//status", "@org_golang_x_exp//maps", - "@org_golang_x_exp//slices", "@org_golang_x_sync//errgroup", "@org_uber_go_atomic//:atomic", "@org_uber_go_multierr//:multierr", diff --git a/br/pkg/lightning/importer/table_import.go b/br/pkg/lightning/importer/table_import.go index 3865ecee635a8..2229f37579963 100644 --- a/br/pkg/lightning/importer/table_import.go +++ b/br/pkg/lightning/importer/table_import.go @@ -15,11 +15,13 @@ package importer import ( + "cmp" "context" "database/sql" "encoding/hex" "fmt" "path/filepath" + "slices" "strings" "sync" "time" @@ -53,7 +55,6 @@ import ( "github.com/pingcap/tidb/util/mathutil" "go.uber.org/multierr" "go.uber.org/zap" - "golang.org/x/exp/slices" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" ) @@ -464,7 +465,7 @@ func (tr *TableImporter) importEngines(pCtx context.Context, rc *Controller, cp for engineID, engine := range cp.Engines { allEngines = append(allEngines, engineCheckpoint{engineID: engineID, checkpoint: engine}) } - slices.SortFunc(allEngines, func(i, j engineCheckpoint) bool { return i.engineID < j.engineID }) + slices.SortFunc(allEngines, func(i, j engineCheckpoint) int { return cmp.Compare(i.engineID, j.engineID) }) for _, ecp := range allEngines { engineID := ecp.engineID diff --git a/br/pkg/lightning/mydump/BUILD.bazel b/br/pkg/lightning/mydump/BUILD.bazel index f037a4fc559b0..200d18c07277f 100644 --- a/br/pkg/lightning/mydump/BUILD.bazel +++ b/br/pkg/lightning/mydump/BUILD.bazel @@ -37,7 +37,6 @@ go_library( "@com_github_xitongsys_parquet_go//parquet", "@com_github_xitongsys_parquet_go//reader", "@com_github_xitongsys_parquet_go//source", - "@org_golang_x_exp//slices", "@org_golang_x_sync//errgroup", "@org_golang_x_text//encoding", "@org_golang_x_text//encoding/charmap", diff --git a/br/pkg/lightning/mydump/csv_parser.go b/br/pkg/lightning/mydump/csv_parser.go index 834a582c271af..ad93b8d885756 100644 --- a/br/pkg/lightning/mydump/csv_parser.go +++ b/br/pkg/lightning/mydump/csv_parser.go @@ -19,6 +19,7 @@ import ( "context" "io" "regexp" + "slices" "strings" "github.com/pingcap/errors" @@ -29,7 +30,6 @@ import ( tidbconfig "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mathutil" - "golang.org/x/exp/slices" ) var ( diff --git a/cmd/mirror/BUILD.bazel b/cmd/mirror/BUILD.bazel index 653de4622abfa..a725799ec05a8 100644 --- a/cmd/mirror/BUILD.bazel +++ b/cmd/mirror/BUILD.bazel @@ -13,7 +13,6 @@ go_library( "@com_google_cloud_go_storage//:storage", "@io_bazel_rules_go//go/tools/bazel:go_default_library", "@org_golang_google_api//googleapi", - "@org_golang_x_exp//slices", "@org_golang_x_sync//errgroup", ], ) diff --git a/cmd/mirror/mirror.go b/cmd/mirror/mirror.go index 78bcac5cd4125..10e98097b6e53 100644 --- a/cmd/mirror/mirror.go +++ b/cmd/mirror/mirror.go @@ -27,12 +27,12 @@ import ( "os" "os/exec" "path/filepath" + "slices" "sort" "strings" "cloud.google.com/go/storage" "github.com/bazelbuild/rules_go/go/tools/bazel" - "golang.org/x/exp/slices" "golang.org/x/sync/errgroup" "google.golang.org/api/googleapi" ) diff --git a/ddl/job_table.go b/ddl/job_table.go index fd1bb28ff0b13..757ebbaa31e22 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -20,6 +20,7 @@ import ( "encoding/hex" "encoding/json" "fmt" + "slices" "strconv" "strings" "time" @@ -44,7 +45,6 @@ import ( "github.com/pingcap/tidb/util/logutil" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" - "golang.org/x/exp/slices" ) var ( diff --git a/executor/aggregate.go b/executor/aggregate.go index 506d5d915de9a..f84a1051083a5 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -16,8 +16,10 @@ package executor import ( "bytes" + "cmp" "context" "fmt" + "slices" "sync" "sync/atomic" "time" @@ -46,7 +48,6 @@ import ( "github.com/pingcap/tidb/util/set" "github.com/twmb/murmur3" "go.uber.org/zap" - "golang.org/x/exp/slices" ) type aggPartialResultMapper map[string][]aggfuncs.PartialResult @@ -1189,7 +1190,7 @@ func (*HashAggRuntimeStats) workerString(buf *bytes.Buffer, prefix string, concu time.Duration(wallTime), concurrency, totalTaskNum, time.Duration(totalWait), time.Duration(totalExec), time.Duration(totalTime)) n := len(workerStats) if n > 0 { - slices.SortFunc(workerStats, func(i, j *AggWorkerStat) bool { return i.WorkerTime < j.WorkerTime }) + slices.SortFunc(workerStats, func(i, j *AggWorkerStat) int { return cmp.Compare(i.WorkerTime, j.WorkerTime) }) fmt.Fprintf(buf, ", max:%v, p95:%v", time.Duration(workerStats[n-1].WorkerTime), time.Duration(workerStats[n*19/20].WorkerTime)) } diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 233d42b521fad..1518abfaf0b8e 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -21,6 +21,7 @@ import ( "encoding/json" "fmt" "math" + "slices" "strconv" "strings" "time" @@ -82,7 +83,6 @@ import ( "github.com/tikv/client-go/v2/tikvrpc" "github.com/tikv/client-go/v2/txnkv/txnlock" "go.uber.org/zap" - "golang.org/x/exp/slices" ) type memtableRetriever struct { @@ -2900,9 +2900,7 @@ func (e *hugeMemTableRetriever) retrieve(ctx context.Context, sctx sessionctx.Co if !e.initialized { is := sctx.GetInfoSchema().(infoschema.InfoSchema) dbs := is.AllSchemas() - slices.SortFunc(dbs, func(i, j *model.DBInfo) bool { - return i.Name.L < j.Name.L - }) + slices.SortFunc(dbs, model.LessDBInfo) e.dbs = dbs e.initialized = true e.rows = make([][]types.Datum, 0, 1024) diff --git a/executor/memtable_reader.go b/executor/memtable_reader.go index 60b34ac0129d3..29f50b4e14554 100644 --- a/executor/memtable_reader.go +++ b/executor/memtable_reader.go @@ -16,12 +16,14 @@ package executor import ( "bytes" + "cmp" "container/heap" "context" "encoding/json" "fmt" "io" "net/http" + "slices" "strings" "sync" "time" @@ -47,7 +49,6 @@ import ( "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/set" - "golang.org/x/exp/slices" "google.golang.org/grpc" "google.golang.org/grpc/credentials" "google.golang.org/grpc/credentials/insecure" @@ -252,7 +253,7 @@ func fetchClusterConfig(sctx sessionctx.Context, nodeTypes, nodeAddrs set.String } items = append(items, item{key: key, val: str}) } - slices.SortFunc(items, func(i, j item) bool { return i.key < j.key }) + slices.SortFunc(items, func(i, j item) int { return cmp.Compare(i.key, j.key) }) var rows [][]types.Datum for _, item := range items { rows = append(rows, types.MakeDatums( @@ -279,7 +280,7 @@ func fetchClusterConfig(sctx sessionctx.Context, nodeTypes, nodeAddrs set.String } results = append(results, result) } - slices.SortFunc(results, func(i, j result) bool { return i.idx < j.idx }) + slices.SortFunc(results, func(i, j result) int { return cmp.Compare(i.idx, j.idx) }) for _, result := range results { finalRows = append(finalRows, result.rows...) } diff --git a/executor/show_stats.go b/executor/show_stats.go index ea9dcfb7b7144..904fc1c935edc 100644 --- a/executor/show_stats.go +++ b/executor/show_stats.go @@ -15,8 +15,10 @@ package executor import ( + "cmp" "context" "fmt" + "slices" "strings" "github.com/pingcap/errors" @@ -28,7 +30,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" "github.com/tikv/client-go/v2/oracle" - "golang.org/x/exp/slices" ) func (e *ShowExec) fetchShowStatsExtended() error { @@ -384,7 +385,7 @@ func stableColsStats(colStats map[int64]*statistics.Column) (cols []*statistics. for _, col := range colStats { cols = append(cols, col) } - slices.SortFunc(cols, func(i, j *statistics.Column) bool { return i.ID < j.ID }) + slices.SortFunc(cols, func(i, j *statistics.Column) int { return cmp.Compare(i.ID, j.ID) }) return } @@ -392,7 +393,7 @@ func stableIdxsStats(idxStats map[int64]*statistics.Index) (idxs []*statistics.I for _, idx := range idxStats { idxs = append(idxs, idx) } - slices.SortFunc(idxs, func(i, j *statistics.Index) bool { return i.ID < j.ID }) + slices.SortFunc(idxs, func(i, j *statistics.Index) int { return cmp.Compare(i.ID, j.ID) }) return } diff --git a/parser/model/model.go b/parser/model/model.go index 1402afabd7983..30dfaa4127408 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -15,6 +15,7 @@ package model import ( "bytes" + "cmp" "encoding/json" "fmt" "strconv" @@ -1682,8 +1683,8 @@ func (db *DBInfo) Copy() *DBInfo { } // LessDBInfo is used for sorting DBInfo by DBInfo.Name. -func LessDBInfo(a *DBInfo, b *DBInfo) bool { - return a.Name.L < b.Name.L +func LessDBInfo(a *DBInfo, b *DBInfo) int { + return cmp.Compare(a.Name.L, b.Name.L) } // CIStr is case insensitive string. diff --git a/store/copr/BUILD.bazel b/store/copr/BUILD.bazel index 52711f3401a64..3edae19193042 100644 --- a/store/copr/BUILD.bazel +++ b/store/copr/BUILD.bazel @@ -63,7 +63,6 @@ go_library( "@com_github_twmb_murmur3//:murmur3", "@org_golang_google_grpc//codes", "@org_golang_google_grpc//status", - "@org_golang_x_exp//slices", "@org_uber_go_zap//:zap", ], ) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 21712bb015194..02a2bd77784eb 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -16,11 +16,13 @@ package copr import ( "bytes" + "cmp" "context" "fmt" "io" "math" "math/rand" + "slices" "strconv" "strings" "sync" @@ -46,7 +48,6 @@ import ( "github.com/tikv/client-go/v2/tikvrpc" "github.com/twmb/murmur3" "go.uber.org/zap" - "golang.org/x/exp/slices" ) const fetchTopoMaxBackoff = 20000 @@ -217,13 +218,13 @@ func balanceBatchCopTaskWithContinuity(storeTaskMap map[uint64]*batchCopTask, ca storeTasks := deepCopyStoreTaskMap(storeTaskMap) // Sort regions by their key ranges. - slices.SortFunc(candidateRegionInfos, func(i, j RegionInfo) bool { + slices.SortFunc(candidateRegionInfos, func(i, j RegionInfo) int { // Special case: Sort empty ranges to the end. if i.Ranges.Len() < 1 || j.Ranges.Len() < 1 { - return i.Ranges.Len() > j.Ranges.Len() + return cmp.Compare(j.Ranges.Len(), i.Ranges.Len()) } // StartKey0 < StartKey1 - return bytes.Compare(i.Ranges.At(0).StartKey, j.Ranges.At(0).StartKey) == -1 + return bytes.Compare(i.Ranges.At(0).StartKey, j.Ranges.At(0).StartKey) }) balanceStart := time.Now() @@ -1219,8 +1220,8 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *backoff.Ba }) } // need to make sure the key ranges is sorted - slices.SortFunc(ranges, func(i, j kv.KeyRange) bool { - return bytes.Compare(i.StartKey, j.StartKey) < 0 + slices.SortFunc(ranges, func(i, j kv.KeyRange) int { + return bytes.Compare(i.StartKey, j.StartKey) }) ret, err := buildBatchCopTasksForNonPartitionedTable(ctx, bo, b.store, NewKeyRanges(ranges), b.req.StoreType, false, 0, false, 0, tiflashcompute.DispatchPolicyInvalid, b.tiflashReplicaReadPolicy, b.appendWarning) return ret, err @@ -1240,8 +1241,8 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *backoff.Ba } } // need to make sure the key ranges is sorted - slices.SortFunc(ranges, func(i, j kv.KeyRange) bool { - return bytes.Compare(i.StartKey, j.StartKey) < 0 + slices.SortFunc(ranges, func(i, j kv.KeyRange) int { + return bytes.Compare(i.StartKey, j.StartKey) }) keyRanges = append(keyRanges, NewKeyRanges(ranges)) } diff --git a/store/helper/BUILD.bazel b/store/helper/BUILD.bazel index 3a90615abd2b0..d3d22715d2fea 100644 --- a/store/helper/BUILD.bazel +++ b/store/helper/BUILD.bazel @@ -23,7 +23,6 @@ go_library( "@com_github_tikv_client_go_v2//tikv", "@com_github_tikv_client_go_v2//tikvrpc", "@com_github_tikv_client_go_v2//txnkv/txnlock", - "@org_golang_x_exp//slices", "@org_uber_go_zap//:zap", ], ) diff --git a/store/helper/helper.go b/store/helper/helper.go index e50175dee42d5..bfc48d661edd1 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -17,6 +17,7 @@ package helper import ( "bufio" "bytes" + "cmp" "context" "encoding/hex" "encoding/json" @@ -25,6 +26,7 @@ import ( "math" "net/http" "net/url" + "slices" "strconv" "strings" "time" @@ -47,7 +49,6 @@ import ( "github.com/tikv/client-go/v2/tikvrpc" "github.com/tikv/client-go/v2/txnkv/txnlock" "go.uber.org/zap" - "golang.org/x/exp/slices" ) // Storage represents a storage that connects TiKV. @@ -719,8 +720,8 @@ func (*Helper) GetTablesInfoWithKeyRange(schemas []*model.DBInfo) []TableInfoWit } } } - slices.SortFunc(tables, func(i, j TableInfoWithKeyRange) bool { - return i.getStartKey() < j.getStartKey() + slices.SortFunc(tables, func(i, j TableInfoWithKeyRange) int { + return cmp.Compare(i.getStartKey(), j.getStartKey()) }) return tables } @@ -733,8 +734,8 @@ func (*Helper) ParseRegionsTableInfos(regionsInfo []*RegionInfo, tables []TableI return tableInfos } // tables is sorted in GetTablesInfoWithKeyRange func - slices.SortFunc(regionsInfo, func(i, j *RegionInfo) bool { - return i.getStartKey() < j.getStartKey() + slices.SortFunc(regionsInfo, func(i, j *RegionInfo) int { + return cmp.Compare(i.getStartKey(), j.getStartKey()) }) idx := 0 diff --git a/store/mockstore/unistore/tikv/BUILD.bazel b/store/mockstore/unistore/tikv/BUILD.bazel index 52578cacbc78a..10bb689f2cfb7 100644 --- a/store/mockstore/unistore/tikv/BUILD.bazel +++ b/store/mockstore/unistore/tikv/BUILD.bazel @@ -55,7 +55,6 @@ go_library( "@com_github_tikv_pd_client//:client", "@org_golang_google_grpc//:grpc", "@org_golang_google_grpc//credentials/insecure", - "@org_golang_x_exp//slices", "@org_uber_go_zap//:zap", ], ) diff --git a/store/mockstore/unistore/tikv/mock_region.go b/store/mockstore/unistore/tikv/mock_region.go index 2e3e4fa01ddb1..7bd9c5085df62 100644 --- a/store/mockstore/unistore/tikv/mock_region.go +++ b/store/mockstore/unistore/tikv/mock_region.go @@ -17,6 +17,7 @@ package tikv import ( "bytes" "context" + "slices" "sync" "sync/atomic" "time" @@ -38,7 +39,6 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/tikv/client-go/v2/oracle" pdclient "github.com/tikv/pd/client" - "golang.org/x/exp/slices" ) // MPPTaskHandlerMap is a map of *cophandler.MPPTaskHandler. @@ -401,9 +401,7 @@ func (rm *MockRegionManager) SplitRegion(req *kvrpcpb.SplitRegionRequest) *kvrpc for _, rawKey := range req.SplitKeys { splitKeys = append(splitKeys, codec.EncodeBytes(nil, rawKey)) } - slices.SortFunc(splitKeys, func(i, j []byte) bool { - return bytes.Compare(i, j) < 0 - }) + slices.SortFunc(splitKeys, bytes.Compare) newRegions, err := rm.splitKeys(splitKeys) if err != nil { diff --git a/timer/api/store.go b/timer/api/store.go index 8055773e65339..0617dd2ebe7cd 100644 --- a/timer/api/store.go +++ b/timer/api/store.go @@ -17,12 +17,12 @@ package api import ( "context" "reflect" + "slices" "strings" "time" "unsafe" "github.com/pingcap/errors" - "golang.org/x/exp/slices" ) type optionalVal interface { From 1c09fe39c5962112f6d619a904eb523b03a2d95f Mon Sep 17 00:00:00 2001 From: CbcWestwolf <1004626265@qq.com> Date: Fri, 18 Aug 2023 17:12:39 +0800 Subject: [PATCH 14/32] *: modify headers of some source files in `privilege` (#46222) --- privilege/conn/conn.go | 2 +- privilege/privileges/ldap/const.go | 2 +- privilege/privileges/ldap/ldap_common.go | 2 +- privilege/privileges/ldap/ldap_common_test.go | 2 +- privilege/privileges/ldap/sasl.go | 2 +- privilege/privileges/ldap/simple.go | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/privilege/conn/conn.go b/privilege/conn/conn.go index 0011c368c03ca..6746c2fcf848a 100644 --- a/privilege/conn/conn.go +++ b/privilege/conn/conn.go @@ -1,4 +1,4 @@ -// Copyright 2023-2023 PingCAP Xingchen (Beijing) Technology Co., Ltd. +// Copyright 2023-2023 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/privilege/privileges/ldap/const.go b/privilege/privileges/ldap/const.go index 5179fd49c07ff..9dfdedd07ccab 100644 --- a/privilege/privileges/ldap/const.go +++ b/privilege/privileges/ldap/const.go @@ -1,4 +1,4 @@ -// Copyright 2023-2023 PingCAP Xingchen (Beijing) Technology Co., Ltd. +// Copyright 2023-2023 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/privilege/privileges/ldap/ldap_common.go b/privilege/privileges/ldap/ldap_common.go index 1b4efd352f07a..48ccaa4a14aaa 100644 --- a/privilege/privileges/ldap/ldap_common.go +++ b/privilege/privileges/ldap/ldap_common.go @@ -1,4 +1,4 @@ -// Copyright 2023-2023 PingCAP Xingchen (Beijing) Technology Co., Ltd. +// Copyright 2023-2023 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/privilege/privileges/ldap/ldap_common_test.go b/privilege/privileges/ldap/ldap_common_test.go index 9a51081b1b423..cc03d2e58422e 100644 --- a/privilege/privileges/ldap/ldap_common_test.go +++ b/privilege/privileges/ldap/ldap_common_test.go @@ -1,4 +1,4 @@ -// Copyright 2023-2023 PingCAP Xingchen (Beijing) Technology Co., Ltd. +// Copyright 2023-2023 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/privilege/privileges/ldap/sasl.go b/privilege/privileges/ldap/sasl.go index f3b81a68be25c..ea3850f58b57c 100644 --- a/privilege/privileges/ldap/sasl.go +++ b/privilege/privileges/ldap/sasl.go @@ -1,4 +1,4 @@ -// Copyright 2023-2023 PingCAP Xingchen (Beijing) Technology Co., Ltd. +// Copyright 2023-2023 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/privilege/privileges/ldap/simple.go b/privilege/privileges/ldap/simple.go index 07934699832a4..d6dabc64eebda 100644 --- a/privilege/privileges/ldap/simple.go +++ b/privilege/privileges/ldap/simple.go @@ -1,4 +1,4 @@ -// Copyright 2023-2023 PingCAP Xingchen (Beijing) Technology Co., Ltd. +// Copyright 2023-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. From e8ca3d4e10320872fd80efc2c12830b6f60eb053 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BA=8C=E6=89=8B=E6=8E=89=E5=8C=85=E5=B7=A5=E7=A8=8B?= =?UTF-8?q?=E5=B8=88?= Date: Fri, 18 Aug 2023 17:12:45 +0800 Subject: [PATCH 15/32] planner: use fixed TblInfoID in `stabilizeGetStatsTblInfo` (#46230) close pingcap/tidb#46229 --- planner/core/debugtrace.go | 2 ++ server/server_test.go | 3 +++ server/testdata/optimizer_suite_out.json | 10 +++++----- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/planner/core/debugtrace.go b/planner/core/debugtrace.go index 13928cfb69fe9..609651b9c554d 100644 --- a/planner/core/debugtrace.go +++ b/planner/core/debugtrace.go @@ -202,7 +202,9 @@ func debugTraceGetStatsTbl( root.AppendStepToCurrentContext(traceInfo) } +// Only for test. func stabilizeGetStatsTblInfo(info *getStatsTblInfo) { + info.TblInfoID = 100 info.InputPhysicalID = 100 tbl := info.StatsTblInfo if tbl == nil { diff --git a/server/server_test.go b/server/server_test.go index 9bfc5d38d906d..0621a27187bbb 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -34,6 +34,9 @@ import ( "github.com/stretchr/testify/require" ) +// cmd: go test -run=^TestOptimizerDebugTrace$ --tags=intest github.com/pingcap/tidb/server +// If you want to update the test result, please run the following command: +// cmd: go test -run=^TestOptimizerDebugTrace$ --tags=intest github.com/pingcap/tidb/server --record func TestOptimizerDebugTrace(t *testing.T) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/planner/SetBindingTimeToZero", `return(true)`)) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/planner/core/DebugTraceStableStatsTbl", `return(true)`)) diff --git a/server/testdata/optimizer_suite_out.json b/server/testdata/optimizer_suite_out.json index 9b28c6e94768b..16b0e6f88ec4a 100644 --- a/server/testdata/optimizer_suite_out.json +++ b/server/testdata/optimizer_suite_out.json @@ -71,7 +71,7 @@ "Version": 440930000000000000 }, "TableName": "t", - "TblInfoID": 98, + "TblInfoID": 100, "Uninitialized": true, "UsePartitionStats": false } @@ -139,7 +139,7 @@ "Version": 440930000000000000 }, "TableName": "t", - "TblInfoID": 98, + "TblInfoID": 100, "Uninitialized": true, "UsePartitionStats": false } @@ -379,7 +379,7 @@ "Version": 440930000000000000 }, "TableName": "t", - "TblInfoID": 98, + "TblInfoID": 100, "Uninitialized": true, "UsePartitionStats": false } @@ -550,7 +550,7 @@ "Version": 440930000000000000 }, "TableName": "t", - "TblInfoID": 98, + "TblInfoID": 100, "Uninitialized": true, "UsePartitionStats": false } @@ -618,7 +618,7 @@ "Version": 440930000000000000 }, "TableName": "t", - "TblInfoID": 98, + "TblInfoID": 100, "Uninitialized": true, "UsePartitionStats": false } From 3e5c7512d51c85df7bf50e28bd87ec42386e89a1 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Fri, 18 Aug 2023 11:54:31 +0200 Subject: [PATCH 16/32] *: Remove and add partitioning (#42907) close pingcap/tidb#42616 --- ddl/column.go | 7 +- ddl/ddl.go | 4 +- ddl/ddl_api.go | 256 +++++++++++-- ddl/ddl_worker.go | 34 +- ddl/delete_range.go | 4 +- ddl/partition.go | 182 ++++++--- ddl/partition_test.go | 100 +++++ ddl/reorg.go | 3 +- ddl/rollingback.go | 3 +- ddl/sanity_check.go | 3 +- ddl/tests/partition/db_partition_test.go | 401 +++++++++++++++++++- executor/batch_point_get.go | 6 +- expression/simple_rewriter.go | 3 + infoschema/builder.go | 10 +- meta/autoid/autoid.go | 2 + parser/model/ddl.go | 7 +- parser/model/ddl_test.go | 2 + parser/model/model.go | 16 + parser/parser_test.go | 13 + planner/core/initialize.go | 4 +- planner/core/point_get_plan.go | 2 +- planner/core/util.go | 13 +- statistics/handle/ddl.go | 38 +- store/gcworker/gc_worker.go | 8 +- table/tables/BUILD.bazel | 2 +- table/tables/partition.go | 170 +++++---- table/tables/partition_test.go | 453 ++++++++++++++++++++--- 27 files changed, 1501 insertions(+), 245 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 8f4b985d9c1e6..03dd5cda82733 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -1070,7 +1070,12 @@ func (w *worker) updatePhysicalTableRow(t table.Table, reorgInfo *reorgInfo) err return dbterror.ErrCancelledDDLJob.GenWithStack("Can not find partition id %d for table %d", reorgInfo.PhysicalTableID, t.Meta().ID) } workType := typeReorgPartitionWorker - if reorgInfo.Job.Type != model.ActionReorganizePartition { + switch reorgInfo.Job.Type { + case model.ActionReorganizePartition, + model.ActionRemovePartitioning, + model.ActionAlterTablePartitioning: + // Expected + default: // workType = typeUpdateColumnWorker // TODO: Support Modify Column on partitioned table // https://github.com/pingcap/tidb/issues/38297 diff --git a/ddl/ddl.go b/ddl/ddl.go index 33fc9247a17bd..52853e10488b2 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -984,7 +984,9 @@ func getIntervalFromPolicy(policy []time.Duration, i int) (time.Duration, bool) func getJobCheckInterval(job *model.Job, i int) (time.Duration, bool) { switch job.Type { case model.ActionAddIndex, model.ActionAddPrimaryKey, model.ActionModifyColumn, - model.ActionReorganizePartition: + model.ActionReorganizePartition, + model.ActionRemovePartitioning, + model.ActionAlterTablePartitioning: return getIntervalFromPolicy(slowDDLIntervalPolicy, i) case model.ActionCreateTable, model.ActionCreateSchema: return getIntervalFromPolicy(fastDDLIntervalPolicy, i) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 2d72c7bd2900c..d456a2757cc36 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3502,6 +3502,7 @@ func ResolveAlterTableSpec(ctx sessionctx.Context, specs []*ast.AlterTableSpec) } else { validSpecs = append(validSpecs, spec) } + // TODO: Only allow REMOVE PARTITIONING as a single ALTER TABLE statement? } // Verify whether the algorithm is supported. @@ -3594,7 +3595,7 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast case ast.AlterTableOptimizePartition: err = errors.Trace(dbterror.ErrUnsupportedOptimizePartition) case ast.AlterTableRemovePartitioning: - err = errors.Trace(dbterror.ErrUnsupportedRemovePartition) + err = d.RemovePartitioning(sctx, ident, spec) case ast.AlterTableRepairPartition: err = errors.Trace(dbterror.ErrUnsupportedRepairPartition) case ast.AlterTableDropColumn: @@ -3671,8 +3672,7 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast isAlterTable := true err = d.renameTable(sctx, ident, newIdent, isAlterTable) case ast.AlterTablePartition: - // Prevent silent succeed if user executes ALTER TABLE x PARTITION BY ... - err = errors.New("alter table partition is unsupported") + err = d.AlterTablePartitioning(sctx, ident, spec) case ast.AlterTableOption: var placementPolicyRef *model.PolicyRefInfo for i, opt := range spec.Options { @@ -4217,11 +4217,12 @@ func getReorganizedDefinitions(pi *model.PartitionInfo, firstPartIdx, lastPartId return tmpDefs } -func getReplacedPartitionIDs(names []model.CIStr, pi *model.PartitionInfo) (firstPartIdx int, lastPartIdx int, idMap map[int]struct{}, err error) { +func getReplacedPartitionIDs(names []string, pi *model.PartitionInfo) (firstPartIdx int, lastPartIdx int, idMap map[int]struct{}, err error) { idMap = make(map[int]struct{}) firstPartIdx, lastPartIdx = -1, -1 for _, name := range names { - partIdx := pi.FindPartitionDefinitionByName(name.L) + nameL := strings.ToLower(name) + partIdx := pi.FindPartitionDefinitionByName(nameL) if partIdx == -1 { return 0, 0, nil, errors.Trace(dbterror.ErrWrongPartitionName) } @@ -4256,6 +4257,86 @@ func getReplacedPartitionIDs(names []model.CIStr, pi *model.PartitionInfo) (firs return firstPartIdx, lastPartIdx, idMap, nil } +func getPartitionInfoTypeNone() *model.PartitionInfo { + return &model.PartitionInfo{ + Type: model.PartitionTypeNone, + Enable: true, + Definitions: []model.PartitionDefinition{{ + Name: model.NewCIStr("pFullTable"), + Comment: "Intermediate partition during ALTER TABLE ... PARTITION BY ...", + }}, + Num: 1, + } +} + +// AlterTablePartitioning reorganize one set of partitions to a new set of partitions. +func (d *ddl) AlterTablePartitioning(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + schema, t, err := d.getSchemaAndTableByIdent(ctx, ident) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.FastGenByArgs(ident.Schema, ident.Name)) + } + + meta := t.Meta().Clone() + piOld := meta.GetPartitionInfo() + var partNames []string + if piOld != nil { + partNames = make([]string, 0, len(piOld.Definitions)) + for i := range piOld.Definitions { + partNames = append(partNames, piOld.Definitions[i].Name.L) + } + } else { + piOld = getPartitionInfoTypeNone() + meta.Partition = piOld + partNames = append(partNames, piOld.Definitions[0].Name.L) + } + newMeta := meta.Clone() + err = buildTablePartitionInfo(ctx, spec.Partition, newMeta) + if err != nil { + return err + } + newPartInfo := newMeta.Partition + + if err = d.assignPartitionIDs(newPartInfo.Definitions); err != nil { + return errors.Trace(err) + } + // A new table ID would be needed for + // the global index, which cannot be the same as the current table id, + // since this table id will be removed in the final state when removing + // all the data with this table id. + var newID []int64 + newID, err = d.genGlobalIDs(1) + if err != nil { + return errors.Trace(err) + } + newPartInfo.NewTableID = newID[0] + newPartInfo.DDLType = piOld.Type + + tzName, tzOffset := ddlutil.GetTimeZone(ctx) + job := &model.Job{ + SchemaID: schema.ID, + TableID: meta.ID, + SchemaName: schema.Name.L, + TableName: t.Meta().Name.L, + Type: model.ActionAlterTablePartitioning, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{partNames, newPartInfo}, + ReorgMeta: &model.DDLReorgMeta{ + SQLMode: ctx.GetSessionVars().SQLMode, + Warnings: make(map[errors.ErrorID]*terror.Error), + WarningsCount: make(map[errors.ErrorID]int64), + Location: &model.TimeZoneLocation{Name: tzName, Offset: tzOffset}, + }, + } + + // No preSplitAndScatter here, it will be done by the worker in onReorganizePartition instead. + err = d.DoDDLJob(ctx, job) + err = d.callHookOnChanged(job, err) + if err == nil { + ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("The statistics of new partitions will be outdated after reorganizing partitions. Please use 'ANALYZE TABLE' statement if you want to update it now")) + } + return errors.Trace(err) +} + // ReorganizePartitions reorganize one set of partitions to a new set of partitions. func (d *ddl) ReorganizePartitions(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { schema, t, err := d.getSchemaAndTableByIdent(ctx, ident) @@ -4278,7 +4359,11 @@ func (d *ddl) ReorganizePartitions(ctx sessionctx.Context, ident ast.Ident, spec default: return errors.Trace(dbterror.ErrUnsupportedReorganizePartition) } - firstPartIdx, lastPartIdx, idMap, err := getReplacedPartitionIDs(spec.PartitionNames, pi) + partNames := make([]string, 0, len(spec.PartitionNames)) + for _, name := range spec.PartitionNames { + partNames = append(partNames, name.L) + } + firstPartIdx, lastPartIdx, idMap, err := getReplacedPartitionIDs(partNames, pi) if err != nil { return errors.Trace(err) } @@ -4289,7 +4374,7 @@ func (d *ddl) ReorganizePartitions(ctx sessionctx.Context, ident ast.Ident, spec if err = d.assignPartitionIDs(partInfo.Definitions); err != nil { return errors.Trace(err) } - if err = checkReorgPartitionDefs(ctx, meta, partInfo, firstPartIdx, lastPartIdx, idMap); err != nil { + if err = checkReorgPartitionDefs(ctx, model.ActionReorganizePartition, meta, partInfo, firstPartIdx, lastPartIdx, idMap); err != nil { return errors.Trace(err) } if err = handlePartitionPlacement(ctx, partInfo); err != nil { @@ -4304,7 +4389,7 @@ func (d *ddl) ReorganizePartitions(ctx sessionctx.Context, ident ast.Ident, spec TableName: t.Meta().Name.L, Type: model.ActionReorganizePartition, BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{spec.PartitionNames, partInfo}, + Args: []interface{}{partNames, partInfo}, ReorgMeta: &model.DDLReorgMeta{ SQLMode: ctx.GetSessionVars().SQLMode, Warnings: make(map[errors.ErrorID]*terror.Error), @@ -4322,55 +4407,137 @@ func (d *ddl) ReorganizePartitions(ctx sessionctx.Context, ident ast.Ident, spec return errors.Trace(err) } -func checkReorgPartitionDefs(ctx sessionctx.Context, tblInfo *model.TableInfo, partInfo *model.PartitionInfo, firstPartIdx, lastPartIdx int, idMap map[int]struct{}) error { +// RemovePartitioning removes partitioning from a table. +func (d *ddl) RemovePartitioning(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + schema, t, err := d.getSchemaAndTableByIdent(ctx, ident) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.FastGenByArgs(ident.Schema, ident.Name)) + } + + meta := t.Meta().Clone() + pi := meta.GetPartitionInfo() + if pi == nil { + return dbterror.ErrPartitionMgmtOnNonpartitioned + } + // TODO: Optimize for remove partitioning with a single partition + // TODO: Add the support for this in onReorganizePartition + // skip if only one partition + // If there are only one partition, then we can do: + // change the table id to the partition id + // and keep the statistics for the partition id (which should be similar to the global statistics) + // and it let the GC clean up the old table metadata including possible global index. + + newSpec := &ast.AlterTableSpec{} + newSpec.Tp = spec.Tp + defs := make([]*ast.PartitionDefinition, 1) + defs[0] = &ast.PartitionDefinition{} + defs[0].Name = model.NewCIStr("CollapsedPartitions") + newSpec.PartDefinitions = defs + partNames := make([]string, len(pi.Definitions)) + for i := range pi.Definitions { + partNames[i] = pi.Definitions[i].Name.L + } + meta.Partition.Type = model.PartitionTypeNone + partInfo, err := BuildAddedPartitionInfo(ctx, meta, newSpec) + if err != nil { + return errors.Trace(err) + } + if err = d.assignPartitionIDs(partInfo.Definitions); err != nil { + return errors.Trace(err) + } + // TODO: check where the default placement comes from (i.e. table level) + if err = handlePartitionPlacement(ctx, partInfo); err != nil { + return errors.Trace(err) + } + partInfo.NewTableID = partInfo.Definitions[0].ID + + tzName, tzOffset := ddlutil.GetTimeZone(ctx) + job := &model.Job{ + SchemaID: schema.ID, + TableID: meta.ID, + SchemaName: schema.Name.L, + TableName: meta.Name.L, + Type: model.ActionRemovePartitioning, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{partNames, partInfo}, + ReorgMeta: &model.DDLReorgMeta{ + SQLMode: ctx.GetSessionVars().SQLMode, + Warnings: make(map[errors.ErrorID]*terror.Error), + WarningsCount: make(map[errors.ErrorID]int64), + Location: &model.TimeZoneLocation{Name: tzName, Offset: tzOffset}, + }, + } + + // No preSplitAndScatter here, it will be done by the worker in onReorganizePartition instead. + err = d.DoDDLJob(ctx, job) + err = d.callHookOnChanged(job, err) + return errors.Trace(err) +} + +func checkReorgPartitionDefs(ctx sessionctx.Context, action model.ActionType, tblInfo *model.TableInfo, partInfo *model.PartitionInfo, firstPartIdx, lastPartIdx int, idMap map[int]struct{}) error { // partInfo contains only the new added partition, we have to combine it with the // old partitions to check all partitions is strictly increasing. pi := tblInfo.Partition clonedMeta := tblInfo.Clone() - clonedMeta.Partition.AddingDefinitions = partInfo.Definitions - clonedMeta.Partition.Definitions = getReorganizedDefinitions(clonedMeta.Partition, firstPartIdx, lastPartIdx, idMap) + switch action { + case model.ActionRemovePartitioning, model.ActionAlterTablePartitioning: + clonedMeta.Partition = partInfo + clonedMeta.ID = partInfo.NewTableID + case model.ActionReorganizePartition: + clonedMeta.Partition.AddingDefinitions = partInfo.Definitions + clonedMeta.Partition.Definitions = getReorganizedDefinitions(clonedMeta.Partition, firstPartIdx, lastPartIdx, idMap) + default: + return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("partition type") + } if err := checkPartitionDefinitionConstraints(ctx, clonedMeta); err != nil { return errors.Trace(err) } - if pi.Type == model.PartitionTypeRange { - if lastPartIdx == len(pi.Definitions)-1 { - // Last partition dropped, OK to change the end range - // Also includes MAXVALUE - return nil - } - // Check if the replaced end range is the same as before - lastAddingPartition := partInfo.Definitions[len(partInfo.Definitions)-1] - lastOldPartition := pi.Definitions[lastPartIdx] - if len(pi.Columns) > 0 { - newGtOld, err := checkTwoRangeColumns(ctx, &lastAddingPartition, &lastOldPartition, pi, tblInfo) + if action == model.ActionReorganizePartition { + if pi.Type == model.PartitionTypeRange { + if lastPartIdx == len(pi.Definitions)-1 { + // Last partition dropped, OK to change the end range + // Also includes MAXVALUE + return nil + } + // Check if the replaced end range is the same as before + lastAddingPartition := partInfo.Definitions[len(partInfo.Definitions)-1] + lastOldPartition := pi.Definitions[lastPartIdx] + if len(pi.Columns) > 0 { + newGtOld, err := checkTwoRangeColumns(ctx, &lastAddingPartition, &lastOldPartition, pi, tblInfo) + if err != nil { + return errors.Trace(err) + } + if newGtOld { + return errors.Trace(dbterror.ErrRangeNotIncreasing) + } + oldGtNew, err := checkTwoRangeColumns(ctx, &lastOldPartition, &lastAddingPartition, pi, tblInfo) + if err != nil { + return errors.Trace(err) + } + if oldGtNew { + return errors.Trace(dbterror.ErrRangeNotIncreasing) + } + return nil + } + + isUnsigned := isPartExprUnsigned(tblInfo) + currentRangeValue, _, err := getRangeValue(ctx, pi.Definitions[lastPartIdx].LessThan[0], isUnsigned) if err != nil { return errors.Trace(err) } - if newGtOld { - return errors.Trace(dbterror.ErrRangeNotIncreasing) - } - oldGtNew, err := checkTwoRangeColumns(ctx, &lastOldPartition, &lastAddingPartition, pi, tblInfo) + newRangeValue, _, err := getRangeValue(ctx, partInfo.Definitions[len(partInfo.Definitions)-1].LessThan[0], isUnsigned) if err != nil { return errors.Trace(err) } - if oldGtNew { + + if currentRangeValue != newRangeValue { return errors.Trace(dbterror.ErrRangeNotIncreasing) } - return nil - } - - isUnsigned := isPartExprUnsigned(tblInfo) - currentRangeValue, _, err := getRangeValue(ctx, pi.Definitions[lastPartIdx].LessThan[0], isUnsigned) - if err != nil { - return errors.Trace(err) } - newRangeValue, _, err := getRangeValue(ctx, partInfo.Definitions[len(partInfo.Definitions)-1].LessThan[0], isUnsigned) - if err != nil { - return errors.Trace(err) - } - - if currentRangeValue != newRangeValue { - return errors.Trace(dbterror.ErrRangeNotIncreasing) + } else { + if len(pi.Definitions) != (lastPartIdx - firstPartIdx + 1) { + // if not ActionReorganizePartition, require all partitions to be changed. + return errors.Trace(dbterror.ErrAlterOperationNotSupported) } } return nil @@ -7487,6 +7654,8 @@ func validateCommentLength(vars *variable.SessionVars, name string, comment *str func BuildAddedPartitionInfo(ctx sessionctx.Context, meta *model.TableInfo, spec *ast.AlterTableSpec) (*model.PartitionInfo, error) { numParts := uint64(0) switch meta.Partition.Type { + case model.PartitionTypeNone: + // OK case model.PartitionTypeList: if len(spec.PartDefinitions) == 0 { return nil, ast.ErrPartitionsMustBeDefined.GenWithStackByArgs(meta.Partition.Type) @@ -7510,6 +7679,10 @@ func BuildAddedPartitionInfo(ctx sessionctx.Context, meta *model.TableInfo, spec } case model.PartitionTypeHash, model.PartitionTypeKey: switch spec.Tp { + case ast.AlterTableRemovePartitioning: + numParts = 1 + default: + return nil, errors.Trace(dbterror.ErrUnsupportedAddPartition) case ast.AlterTableCoalescePartitions: if int(spec.Num) >= len(meta.Partition.Definitions) { return nil, dbterror.ErrDropLastPartition @@ -7540,6 +7713,7 @@ func BuildAddedPartitionInfo(ctx sessionctx.Context, meta *model.TableInfo, spec } part.Definitions = defs + part.Num = uint64(len(defs)) return part, nil } diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index f055aaaeff34a..2417500a0fe63 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -546,7 +546,8 @@ func jobNeedGC(job *model.Job) bool { model.ActionDropTablePartition, model.ActionTruncateTablePartition, model.ActionDropColumn, model.ActionModifyColumn, model.ActionAddIndex, model.ActionAddPrimaryKey, - model.ActionReorganizePartition: + model.ActionReorganizePartition, model.ActionRemovePartitioning, + model.ActionAlterTablePartitioning: return true case model.ActionMultiSchemaChange: for _, sub := range job.MultiSchemaInfo.SubJobs { @@ -1125,7 +1126,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, ver, err = w.onFlashbackCluster(d, t, job) case model.ActionMultiSchemaChange: ver, err = onMultiSchemaChange(w, d, t, job) - case model.ActionReorganizePartition: + case model.ActionReorganizePartition, model.ActionRemovePartitioning, + model.ActionAlterTablePartitioning: ver, err = w.onReorganizePartition(d, t, job) case model.ActionAlterTTLInfo: ver, err = onTTLInfoChange(d, t, job) @@ -1425,6 +1427,7 @@ func updateSchemaVersion(d *ddlCtx, t *meta.Meta, job *model.Job, multiInfos ... } case model.ActionReorganizePartition: diff.TableID = job.TableID + // TODO: should this be for every state of Reorganize? if len(job.CtxVars) > 0 { if droppedIDs, ok := job.CtxVars[0].([]int64); ok { if addedIDs, ok := job.CtxVars[1].([]int64); ok { @@ -1439,6 +1442,33 @@ func updateSchemaVersion(d *ddlCtx, t *meta.Meta, job *model.Job, multiInfos ... } } } + case model.ActionRemovePartitioning, model.ActionAlterTablePartitioning: + diff.TableID = job.TableID + diff.OldTableID = job.TableID + if job.SchemaState == model.StateDeleteReorganization { + partInfo := &model.PartitionInfo{} + var partNames []string + err = job.DecodeArgs(&partNames, &partInfo) + if err != nil { + return 0, errors.Trace(err) + } + // Final part, new table id is assigned + diff.TableID = partInfo.NewTableID + if len(job.CtxVars) > 0 { + if droppedIDs, ok := job.CtxVars[0].([]int64); ok { + if addedIDs, ok := job.CtxVars[1].([]int64); ok { + // to use AffectedOpts we need both new and old to have the same length + maxParts := mathutil.Max[int](len(droppedIDs), len(addedIDs)) + // Also initialize them to 0! + oldIDs := make([]int64, maxParts) + copy(oldIDs, droppedIDs) + newIDs := make([]int64, maxParts) + copy(newIDs, addedIDs) + diff.AffectedOpts = buildPlacementAffects(oldIDs, newIDs) + } + } + } + } case model.ActionCreateTable: diff.TableID = job.TableID if len(job.Args) > 0 { diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 6d92fdbefded0..e402ac89a7c69 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -313,7 +313,9 @@ func insertJobIntoDeleteRangeTable(ctx context.Context, sctx sessionctx.Context, endKey := tablecodec.EncodeTablePrefix(tableID + 1) elemID := ea.allocForPhysicalID(tableID) return doInsert(ctx, s, job.ID, elemID, startKey, endKey, now, fmt.Sprintf("table ID is %d", tableID)) - case model.ActionDropTablePartition, model.ActionTruncateTablePartition, model.ActionReorganizePartition: + case model.ActionDropTablePartition, model.ActionTruncateTablePartition, + model.ActionReorganizePartition, model.ActionRemovePartitioning, + model.ActionAlterTablePartitioning: var physicalTableIDs []int64 // partInfo is not used, but is set in ReorgPartition. // Better to have an additional argument in job.DecodeArgs since it is ignored, diff --git a/ddl/partition.go b/ddl/partition.go index 96f01e02828e7..c5bbe4b12d64a 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -90,6 +90,7 @@ func checkAddPartition(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.P return tblInfo, partInfo, []model.PartitionDefinition{}, nil } +// TODO: Move this into reorganize partition! func (w *worker) onAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { // Handle the rolling back job if job.IsRollingback() { @@ -1155,12 +1156,22 @@ func GeneratePartDefsFromInterval(ctx sessionctx.Context, tp ast.AlterTableType, // buildPartitionDefinitionsInfo build partition definitions info without assign partition id. tbInfo will be constant func buildPartitionDefinitionsInfo(ctx sessionctx.Context, defs []*ast.PartitionDefinition, tbInfo *model.TableInfo, numParts uint64) (partitions []model.PartitionDefinition, err error) { switch tbInfo.Partition.Type { + case model.PartitionTypeNone: + if len(defs) != 1 { + return nil, dbterror.ErrUnsupportedPartitionType + } + partitions = []model.PartitionDefinition{{Name: defs[0].Name}} + if comment, set := defs[0].Comment(); set { + partitions[0].Comment = comment + } case model.PartitionTypeRange: partitions, err = buildRangePartitionDefinitions(ctx, defs, tbInfo) case model.PartitionTypeHash, model.PartitionTypeKey: partitions, err = buildHashPartitionDefinitions(ctx, defs, tbInfo, numParts) case model.PartitionTypeList: partitions, err = buildListPartitionDefinitions(ctx, defs, tbInfo) + default: + err = dbterror.ErrUnsupportedPartitionType } if err != nil { @@ -1454,17 +1465,18 @@ func checkAddPartitionNameUnique(tbInfo *model.TableInfo, pi *model.PartitionInf return nil } -func checkReorgPartitionNames(p *model.PartitionInfo, droppedNames []model.CIStr, pi *model.PartitionInfo) error { +func checkReorgPartitionNames(p *model.PartitionInfo, droppedNames []string, pi *model.PartitionInfo) error { partNames := make(map[string]struct{}) oldDefs := p.Definitions for _, oldDef := range oldDefs { partNames[oldDef.Name.L] = struct{}{} } for _, delName := range droppedNames { - if _, ok := partNames[delName.L]; !ok { + droppedName := strings.ToLower(delName) + if _, ok := partNames[droppedName]; !ok { return dbterror.ErrSameNamePartition.GenWithStackByArgs(delName) } - delete(partNames, delName.L) + delete(partNames, droppedName) } newDefs := pi.Definitions for _, newDef := range newDefs { @@ -1883,7 +1895,8 @@ func dropLabelRules(_ *ddlCtx, schemaName, tableName string, partNames []string) // onDropTablePartition deletes old partition meta. func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { var partNames []string - if err := job.DecodeArgs(&partNames); err != nil { + partInfo := model.PartitionInfo{} + if err := job.DecodeArgs(&partNames, &partInfo); err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } @@ -1891,16 +1904,18 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( if err != nil { return ver, errors.Trace(err) } - if job.Type == model.ActionAddTablePartition || job.Type == model.ActionReorganizePartition { + if job.Type == model.ActionAddTablePartition || job.Type == model.ActionReorganizePartition || + job.Type == model.ActionRemovePartitioning || job.Type == model.ActionAlterTablePartitioning { // It is rollback from reorganize partition, just remove DroppingDefinitions from tableInfo tblInfo.Partition.DroppingDefinitions = nil - // It is rollback from adding table partition, just remove addingDefinitions from tableInfo. + // It is rollbacked from adding table partition, just remove addingDefinitions from tableInfo. physicalTableIDs, pNames, rollbackBundles := rollbackAddingPartitionInfo(tblInfo) err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), rollbackBundles) if err != nil { job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to notify PD the placement rules") } + // TODO: Will this drop LabelRules for existing partitions, if the new partitions have the same name? err = dropLabelRules(d, job.SchemaName, tblInfo.Name.L, pNames) if err != nil { job.State = model.JobStateCancelled @@ -1911,6 +1926,19 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( job.State = model.JobStateCancelled return ver, err } + if partInfo.DDLType != model.PartitionTypeNone { + // Also remove anything with the new table id + physicalTableIDs = append(physicalTableIDs, tblInfo.Partition.NewTableID) + // Reset if it was normal table before + if tblInfo.Partition.Type == model.PartitionTypeNone { + tblInfo.Partition = nil + } else { + tblInfo.Partition.NewTableID = 0 + tblInfo.Partition.DDLExpr = "" + tblInfo.Partition.DDLColumns = nil + tblInfo.Partition.DDLType = model.PartitionTypeNone + } + } ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) if err != nil { @@ -2604,21 +2632,35 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, nil } -func checkReorgPartition(t *meta.Meta, job *model.Job) (*model.TableInfo, []model.CIStr, *model.PartitionInfo, []model.PartitionDefinition, []model.PartitionDefinition, error) { +func getReorgPartitionInfo(t *meta.Meta, job *model.Job) (*model.TableInfo, []string, *model.PartitionInfo, []model.PartitionDefinition, []model.PartitionDefinition, error) { schemaID := job.SchemaID tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, schemaID) if err != nil { return nil, nil, nil, nil, nil, errors.Trace(err) } partInfo := &model.PartitionInfo{} - var partNames []model.CIStr + var partNames []string err = job.DecodeArgs(&partNames, &partInfo) if err != nil { job.State = model.JobStateCancelled return nil, nil, nil, nil, nil, errors.Trace(err) } - addingDefs := tblInfo.Partition.AddingDefinitions - droppingDefs := tblInfo.Partition.DroppingDefinitions + var addingDefs, droppingDefs []model.PartitionDefinition + if tblInfo.Partition != nil { + addingDefs = tblInfo.Partition.AddingDefinitions + droppingDefs = tblInfo.Partition.DroppingDefinitions + tblInfo.Partition.NewTableID = partInfo.NewTableID + tblInfo.Partition.DDLType = partInfo.Type + tblInfo.Partition.DDLExpr = partInfo.Expr + tblInfo.Partition.DDLColumns = partInfo.Columns + } else { + tblInfo.Partition = getPartitionInfoTypeNone() + tblInfo.Partition.NewTableID = partInfo.NewTableID + tblInfo.Partition.Definitions[0].ID = tblInfo.ID + tblInfo.Partition.DDLType = partInfo.Type + tblInfo.Partition.DDLExpr = partInfo.Expr + tblInfo.Partition.DDLColumns = partInfo.Columns + } if len(addingDefs) == 0 { addingDefs = []model.PartitionDefinition{} } @@ -2638,18 +2680,11 @@ func (w *worker) onReorganizePartition(d *ddlCtx, t *meta.Meta, job *model.Job) return ver, nil } - tblInfo, partNamesCIStr, partInfo, _, addingDefinitions, err := checkReorgPartition(t, job) + tblInfo, partNames, partInfo, _, addingDefinitions, err := getReorgPartitionInfo(t, job) if err != nil { return ver, err } - partNames := make([]string, len(partNamesCIStr)) - for i := range partNamesCIStr { - partNames[i] = partNamesCIStr[i].L - } - // In order to skip maintaining the state check in partitionDefinition, TiDB use dropping/addingDefinition instead of state field. - // So here using `job.SchemaState` to judge what the stage of this job is. - originalState := job.SchemaState switch job.SchemaState { case model.StateNone: // job.SchemaState == model.StateNone means the job is in the initial state of reorg partition. @@ -2659,28 +2694,27 @@ func (w *worker) onReorganizePartition(d *ddlCtx, t *meta.Meta, job *model.Job) // The partInfo may have been checked against an older schema version for example. // If the check is done here, it does not need to be repeated, since no other // DDL on the same table can be run concurrently. - err = checkAddPartitionTooManyPartitions(uint64(len(tblInfo.Partition.Definitions) + - len(partInfo.Definitions) - - len(partNames))) + num := len(partInfo.Definitions) - len(partNames) + len(tblInfo.Partition.Definitions) + err = checkAddPartitionTooManyPartitions(uint64(num)) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } - err = checkReorgPartitionNames(tblInfo.Partition, partNamesCIStr, partInfo) + err = checkReorgPartitionNames(tblInfo.Partition, partNames, partInfo) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } // Re-check that the dropped/added partitions are compatible with current definition - firstPartIdx, lastPartIdx, idMap, err := getReplacedPartitionIDs(partNamesCIStr, tblInfo.Partition) + firstPartIdx, lastPartIdx, idMap, err := getReplacedPartitionIDs(partNames, tblInfo.Partition) if err != nil { job.State = model.JobStateCancelled return ver, err } sctx := w.sess.Context - if err = checkReorgPartitionDefs(sctx, tblInfo, partInfo, firstPartIdx, lastPartIdx, idMap); err != nil { + if err = checkReorgPartitionDefs(sctx, job.Type, tblInfo, partInfo, firstPartIdx, lastPartIdx, idMap); err != nil { job.State = model.JobStateCancelled return ver, err } @@ -2809,18 +2843,18 @@ func (w *worker) onReorganizePartition(d *ddlCtx, t *meta.Meta, job *model.Job) } } - job.SchemaState = model.StateWriteOnly tblInfo.Partition.DDLState = model.StateWriteOnly metrics.GetBackfillProgressByLabel(metrics.LblReorgPartition, job.SchemaName, tblInfo.Name.String()).Set(0.2 / float64(math.MaxUint64)) - ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != job.SchemaState) + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) + job.SchemaState = model.StateWriteOnly case model.StateWriteOnly: // Insert this state to confirm all servers can see the new partitions when reorg is running, // so that new data will be updated in both old and new partitions when reorganizing. job.SnapshotVer = 0 - job.SchemaState = model.StateWriteReorganization tblInfo.Partition.DDLState = model.StateWriteReorganization metrics.GetBackfillProgressByLabel(metrics.LblReorgPartition, job.SchemaName, tblInfo.Name.String()).Set(0.3 / float64(math.MaxUint64)) - ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != job.SchemaState) + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) + job.SchemaState = model.StateWriteReorganization case model.StateWriteReorganization: physicalTableIDs := getPartitionIDsFromDefinitions(tblInfo.Partition.DroppingDefinitions) tbl, err2 := getTable(d.store, job.SchemaID, tblInfo) @@ -2840,7 +2874,7 @@ func (w *worker) onReorganizePartition(d *ddlCtx, t *meta.Meta, job *model.Job) return ver, err } - firstPartIdx, lastPartIdx, idMap, err2 := getReplacedPartitionIDs(partNamesCIStr, tblInfo.Partition) + firstPartIdx, lastPartIdx, idMap, err2 := getReplacedPartitionIDs(partNames, tblInfo.Partition) failpoint.Inject("reorgPartWriteReorgReplacedPartIDsFail", func(val failpoint.Value) { if val.(bool) { err2 = errors.New("Injected error by reorgPartWriteReorgReplacedPartIDsFail") @@ -2851,16 +2885,22 @@ func (w *worker) onReorganizePartition(d *ddlCtx, t *meta.Meta, job *model.Job) } newDefs := getReorganizedDefinitions(tblInfo.Partition, firstPartIdx, lastPartIdx, idMap) - // From now on, use the new definitions, but keep the Adding and Dropping for double write + // From now on, use the new partitioning, but keep the Adding and Dropping for double write tblInfo.Partition.Definitions = newDefs tblInfo.Partition.Num = uint64(len(newDefs)) + if job.Type == model.ActionAlterTablePartitioning || + job.Type == model.ActionRemovePartitioning { + tblInfo.Partition.Type, tblInfo.Partition.DDLType = tblInfo.Partition.DDLType, tblInfo.Partition.Type + tblInfo.Partition.Expr, tblInfo.Partition.DDLExpr = tblInfo.Partition.DDLExpr, tblInfo.Partition.Expr + tblInfo.Partition.Columns, tblInfo.Partition.DDLColumns = tblInfo.Partition.DDLColumns, tblInfo.Partition.Columns + } // Now all the data copying is done, but we cannot simply remove the droppingDefinitions // since they are a part of the normal Definitions that other nodes with // the current schema version. So we need to double write for one more schema version - job.SchemaState = model.StateDeleteReorganization tblInfo.Partition.DDLState = model.StateDeleteReorganization - ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != job.SchemaState) + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) + job.SchemaState = model.StateDeleteReorganization case model.StateDeleteReorganization: // Drop the droppingDefinitions and finish the DDL @@ -2876,10 +2916,59 @@ func (w *worker) onReorganizePartition(d *ddlCtx, t *meta.Meta, job *model.Job) // and the addingDefinitions for handling in the updateSchemaVersion physicalTableIDs := getPartitionIDsFromDefinitions(tblInfo.Partition.DroppingDefinitions) newIDs := getPartitionIDsFromDefinitions(partInfo.Definitions) - job.CtxVars = []interface{}{physicalTableIDs, newIDs} - definitionsToAdd := tblInfo.Partition.AddingDefinitions + statisticsPartInfo := &model.PartitionInfo{Definitions: tblInfo.Partition.AddingDefinitions} + tblInfo.Partition.DroppingDefinitions = nil tblInfo.Partition.AddingDefinitions = nil + tblInfo.Partition.DDLState = model.StateNone + + if job.Type != model.ActionReorganizePartition { + // ALTER TABLE ... PARTITION BY + // REMOVE PARTITIONING + // New Table ID, so needs to recreate the table by drop+create. + oldTblID := tblInfo.ID + // Overloading the NewTableID here with the oldTblID instead, + // for keeping the old global statistics + statisticsPartInfo.NewTableID = oldTblID + err = t.DropTableOrView(job.SchemaID, tblInfo.ID) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + // TODO: Handle bundles? + // TODO: How to carrie over AUTO_INCREMENT etc.? + // Check if they are carried over in ApplyDiff?!? + err = t.GetAutoIDAccessors(job.SchemaID, tblInfo.ID).Del() + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + tblInfo.ID = partInfo.NewTableID + if partInfo.DDLType != model.PartitionTypeNone { + // if partitioned before, then also add the old table ID, + // otherwise it will be the already included first partition + physicalTableIDs = append(physicalTableIDs, oldTblID) + } + if job.Type == model.ActionRemovePartitioning { + tblInfo.Partition = nil + } else { + // ALTER TABLE ... PARTITION BY + //tblInfo.Partition.Type = tblInfo.Partition.DDLType + //tblInfo.Partition.Expr = tblInfo.Partition.DDLExpr + //tblInfo.Partition.Columns = tblInfo.Partition.DDLColumns + tblInfo.Partition.DDLType = model.PartitionTypeNone + tblInfo.Partition.DDLExpr = "" + tblInfo.Partition.DDLColumns = nil + tblInfo.Partition.NewTableID = 0 + } + // TODO: Add failpoint here? + err = t.CreateTableOrView(job.SchemaID, tblInfo) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + } + job.CtxVars = []interface{}{physicalTableIDs, newIDs} ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) failpoint.Inject("reorgPartWriteReorgSchemaVersionUpdateFail", func(val failpoint.Value) { if val.(bool) { @@ -2889,13 +2978,13 @@ func (w *worker) onReorganizePartition(d *ddlCtx, t *meta.Meta, job *model.Job) if err != nil { return ver, errors.Trace(err) } - job.SchemaState = model.StateNone - tblInfo.Partition.DDLState = model.StateNone job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) // How to handle this? // Seems to only trigger asynchronous update of statistics. // Should it actually be synchronous? - asyncNotifyEvent(d, &util.Event{Tp: model.ActionReorganizePartition, TableInfo: tblInfo, PartInfo: &model.PartitionInfo{Definitions: definitionsToAdd}}) + // Include the old table ID, if changed, which may contain global statistics, + // so it can be reused for the new (non)partitioned table. + asyncNotifyEvent(d, &util.Event{Tp: job.Type, TableInfo: tblInfo, PartInfo: statisticsPartInfo}) // A background job will be created to delete old partition data. job.Args = []interface{}{physicalTableIDs} @@ -2974,22 +3063,19 @@ func newReorgPartitionWorker(sessCtx sessionctx.Context, i int, t table.Physical if pt == nil { return nil, dbterror.ErrUnsupportedReorganizePartition.GenWithStackByArgs() } - partColIDs := pt.GetPartitionColumnIDs() + partColIDs := reorgedTbl.GetPartitionColumnIDs() writeColOffsetMap := make(map[int64]int, len(partColIDs)) maxOffset := 0 - for _, col := range pt.Cols() { - found := false - for _, id := range partColIDs { + for _, id := range partColIDs { + var offset int + for _, col := range pt.Cols() { if col.ID == id { - found = true + offset = col.Offset break } } - if !found { - continue - } - writeColOffsetMap[col.ID] = col.Offset - maxOffset = mathutil.Max[int](maxOffset, col.Offset) + writeColOffsetMap[id] = offset + maxOffset = mathutil.Max[int](maxOffset, offset) } return &reorgPartitionWorker{ backfillCtx: newBackfillCtx(reorgInfo.d, i, sessCtx, reorgInfo.SchemaName, t, jc, "reorg_partition_rate", false), @@ -3079,6 +3165,8 @@ func (w *reorgPartitionWorker) fetchRowColVals(txn kv.Transaction, taskRange reo return false, nil } + // TODO: Extend for normal tables + // TODO: Extend for REMOVE PARTITIONING _, err := w.rowDecoder.DecodeTheExistedColumnMap(w.sessCtx, handle, rawRow, sysTZ, w.rowMap) if err != nil { return false, errors.Trace(err) diff --git a/ddl/partition_test.go b/ddl/partition_test.go index 325fe6a1795da..d345454a14e80 100644 --- a/ddl/partition_test.go +++ b/ddl/partition_test.go @@ -15,6 +15,7 @@ package ddl_test import ( + "fmt" "testing" "time" @@ -246,3 +247,102 @@ func TestReorganizePartitionRollback(t *testing.T) { // test then add index should success tk.MustExec("alter table t1 add index idx_kc (k, c)") } + +func TestAlterPartitionBy(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create schema AlterPartitionBy") + tk.MustExec("use AlterPartitionBy") + // Just for debug... + //tk.MustExec(`create table t (a int primary key, b varchar(255), key (b)) partition by range (a) (partition p0 values less than (10))`) + // First easy example non-partitioned -> partitioned + tk.MustExec(`create table t (a int primary key, b varchar(255), key (b))`) + for i := 0; i < 1000; i++ { + tk.MustExec(fmt.Sprintf(`insert into t values (%d,'filler%d')`, i, i/3)) + } + tk.MustExec(`alter table t partition by range (a) (partition p0 values less than (1000000), partition p1 values less than (2000000), partition pMax values less than (maxvalue))`) + tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 The statistics of new partitions will be outdated after reorganizing partitions. Please use 'ANALYZE TABLE' statement if you want to update it now")) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (1000000),\n" + + " PARTITION `p1` VALUES LESS THAN (2000000),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + tk.MustExec(`alter table t partition by hash(a) partitions 7`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY HASH (`a`) PARTITIONS 7")) + tk.MustExec(`alter table t partition by key(a) partitions 5`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY KEY (`a`) PARTITIONS 5")) +} + +func TestReorgRangeTimestampMaxvalue(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create schema AlterPartitionBy") + tk.MustExec("use AlterPartitionBy") + tk.MustExec(`CREATE TABLE t1 ( +a timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, +b varchar(10), +PRIMARY KEY (a) +) +PARTITION BY RANGE (UNIX_TIMESTAMP(a)) ( +PARTITION p1 VALUES LESS THAN (1199134800), +PARTITION pmax VALUES LESS THAN MAXVALUE +)`) + + tk.MustExec(`ALTER TABLE t1 REORGANIZE PARTITION pmax INTO ( +PARTITION p3 VALUES LESS THAN (1247688000), +PARTITION pmax VALUES LESS THAN MAXVALUE)`) +} + +func TestRemovePartitioningSinglePartition(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + dbName := "RemovePartitioning" + tk.MustExec("create schema " + dbName) + tk.MustExec("use " + dbName) + tk.MustExec(`CREATE TABLE t ( +a int NOT NULL primary key , +b varchar(100), +key (b) +) +PARTITION BY hash (a) PARTITIONS 1`) + tk.MustExec(`insert into t values (1,"a"),(2,"bye"),(3,"Hi")`) + + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(100) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY HASH (`a`) PARTITIONS 1")) + + tk.MustExec(`ALTER TABLE t REMOVE PARTITIONING`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL,\n" + + " `b` varchar(100) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows("1 a", "2 bye", "3 Hi")) +} diff --git a/ddl/reorg.go b/ddl/reorg.go index 1fd2be7e099ce..1abda7c828e47 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -398,7 +398,8 @@ func updateBackfillProgress(w *worker, reorgInfo *reorgInfo, tblInfo *model.Tabl metrics.GetBackfillProgressByLabel(label, reorgInfo.SchemaName, tblInfo.Name.String()).Set(progress * 100) case model.ActionModifyColumn: metrics.GetBackfillProgressByLabel(metrics.LblModifyColumn, reorgInfo.SchemaName, tblInfo.Name.String()).Set(progress * 100) - case model.ActionReorganizePartition: + case model.ActionReorganizePartition, model.ActionRemovePartitioning, + model.ActionAlterTablePartitioning: metrics.GetBackfillProgressByLabel(metrics.LblReorgPartition, reorgInfo.SchemaName, tblInfo.Name.String()).Set(progress * 100) } } diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 3af166b498f80..f6390cae6a63f 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -424,7 +424,8 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) ver, err = rollingbackAddIndex(w, d, t, job, true) case model.ActionAddTablePartition: ver, err = rollingbackAddTablePartition(d, t, job) - case model.ActionReorganizePartition: + case model.ActionReorganizePartition, model.ActionRemovePartitioning, + model.ActionAlterTablePartitioning: ver, err = rollingbackReorganizePartition(d, t, job) case model.ActionDropColumn: ver, err = rollingbackDropColumn(d, t, job) diff --git a/ddl/sanity_check.go b/ddl/sanity_check.go index 8dbdb640a2b09..01b78f1dd04f0 100644 --- a/ddl/sanity_check.go +++ b/ddl/sanity_check.go @@ -100,7 +100,8 @@ func expectedDeleteRangeCnt(ctx delRangeCntCtx, job *model.Job) (int, error) { } return len(physicalTableIDs) + 1, nil case model.ActionDropTablePartition, model.ActionTruncateTablePartition, - model.ActionReorganizePartition: + model.ActionReorganizePartition, model.ActionRemovePartitioning, + model.ActionAlterTablePartitioning: var physicalTableIDs []int64 if err := job.DecodeArgs(&physicalTableIDs); err != nil { return 0, errors.Trace(err) diff --git a/ddl/tests/partition/db_partition_test.go b/ddl/tests/partition/db_partition_test.go index aa7787222466f..f2695a8a4f23b 100644 --- a/ddl/tests/partition/db_partition_test.go +++ b/ddl/tests/partition/db_partition_test.go @@ -4347,7 +4347,6 @@ func TestPartitionErrorCode(t *testing.T) { tk.MustGetErrCode("alter table t_part check partition p0, p1;", errno.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t_part optimize partition p0,p1;", errno.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t_part rebuild partition p0,p1;", errno.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t_part remove partitioning;", errno.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t_part repair partition p1;", errno.ErrUnsupportedDDLOperation) // Reduce the impact on DML when executing partition DDL @@ -4488,6 +4487,17 @@ func TestAddHashPartition(t *testing.T) { ) partition by hash(store_id) partitions 4`) + tk.MustExec(`insert into t values (1,"one", "two","1999-01-04", null, 1, 1), (2, "Who", "Else", '2020-12-02', '2022-12-31', 2, 2)`) + tk.MustExec(`insert into t select id + 2, fname, lname, hired, separated, job_code, store_id + 2 from t`) + tk.MustExec(`insert into t select id + 4, fname, lname, hired, separated, job_code, store_id + 4 from t`) + tk.MustExec(`insert into t select id + 8, fname, lname, hired, separated, job_code, store_id + 8 from t`) + tk.MustExec(`insert into t select id + 16, fname, lname, hired, separated, job_code, store_id + 16 from t`) + tk.MustExec(`insert into t select id + 32, fname, lname, hired, separated, job_code, store_id + 32 from t`) + tk.MustExec(`insert into t select id + 64, fname, lname, hired, separated, job_code, store_id + 64 from t`) + tk.MustExec(`insert into t select id + 128, fname, lname, hired, separated, job_code, store_id + 128 from t`) + tk.MustExec(`insert into t select id + 256, fname, lname, hired, separated, job_code, store_id + 256 from t`) + tk.MustExec(`insert into t select id + 512, fname, lname, hired, separated, job_code, store_id + 512 from t`) + tk.MustQuery(`select sum(store_id), avg(store_id), max(store_id), min(store_id), sum(id) from t`).Check(testkit.Rows("524800 512.5000 1024 1 524800")) // TiDB does not support system versioned tables / SYSTEM_TIME // also the error is slightly wrong with 'VALUES HISTORY' // instead of just 'HISTORY' @@ -4495,6 +4505,7 @@ func TestAddHashPartition(t *testing.T) { tk.MustContainErrMsg(`alter table t add partition (partition pList values in (22))`, "[ddl:1480]Only LIST PARTITIONING can use VALUES IN in partition definition") tk.MustContainErrMsg(`alter table t add partition (partition pRange values less than (22))`, "[ddl:1480]Only RANGE PARTITIONING can use VALUES LESS THAN in partition definition") tk.MustExec(`insert into t values (20, "Joe", "Doe", '2020-01-05', null, 1,1), (21, "Jane", "Doe", '2021-07-05', null, 2,1)`) + tk.MustQuery(`select sum(store_id), avg(store_id), max(store_id), min(store_id), sum(id) from t`).Check(testkit.Rows("524802 511.5029 1024 1 524841")) tk.MustExec("alter table t add partition partitions 8") tk.MustQuery(`show create table t`).Check(testkit.Rows("" + "t CREATE TABLE `t` (\n" + @@ -4588,6 +4599,47 @@ func TestAddHashPartition(t *testing.T) { " `store_id` int(11) DEFAULT NULL\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + "PARTITION BY HASH (`store_id`) PARTITIONS 12")) + tk.MustExec(`alter table t add partition (partition p12 comment 'p12' placement policy tworeplicas)`) + tk.MustExec("alter table t placement policy fourreplicas") + tk.MustExec(`alter table t add partition partitions 1`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `id` int(11) NOT NULL,\n" + + " `fname` varchar(30) DEFAULT NULL,\n" + + " `lname` varchar(30) DEFAULT NULL,\n" + + " `hired` date NOT NULL DEFAULT '1970-01-01',\n" + + " `separated` date DEFAULT NULL,\n" + + " `job_code` int(11) DEFAULT NULL,\n" + + " `store_id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`fourreplicas` */\n" + + "PARTITION BY HASH (`store_id`)\n" + + "(PARTITION `p0`,\n" + + " PARTITION `p1`,\n" + + " PARTITION `p2`,\n" + + " PARTITION `p3`,\n" + + " PARTITION `p4`,\n" + + " PARTITION `p5`,\n" + + " PARTITION `p6`,\n" + + " PARTITION `p7`,\n" + + " PARTITION `p8`,\n" + + " PARTITION `p9`,\n" + + " PARTITION `p10`,\n" + + " PARTITION `p11`,\n" + + " PARTITION `p12` COMMENT 'p12' /*T![placement] PLACEMENT POLICY=`tworeplicas` */,\n" + + " PARTITION `p13`)")) + tk.MustExec(`alter table t remove partitioning`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `id` int(11) NOT NULL,\n" + + " `fname` varchar(30) DEFAULT NULL,\n" + + " `lname` varchar(30) DEFAULT NULL,\n" + + " `hired` date NOT NULL DEFAULT '1970-01-01',\n" + + " `separated` date DEFAULT NULL,\n" + + " `job_code` int(11) DEFAULT NULL,\n" + + " `store_id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`fourreplicas` */")) + tk.MustQuery(`select sum(store_id), avg(store_id), max(store_id), min(store_id), sum(id) from t`).Check(testkit.Rows("524802 511.5029 1024 1 524841")) + tk.MustExec("alter table t placement policy default") tk.MustExec(`drop placement policy tworeplicas`) tk.MustExec(`drop placement policy threereplicas`) tk.MustExec(`drop placement policy fourreplicas`) @@ -4793,8 +4845,21 @@ func TestUnsupportedPartitionManagementDDLs(t *testing.T) { ); `) - _, err := tk.Exec("alter table test_1465 partition by hash(a)") - require.Regexp(t, ".*alter table partition is unsupported", err.Error()) + tk.MustExec(`alter table test_1465 truncate partition p1`) + tk.MustContainErrMsg(`alter table test_1465 check partition p1`, "[ddl:8200]Unsupported check partition") + tk.MustContainErrMsg(`alter table test_1465 optimize partition p1`, "[ddl:8200]Unsupported optimize partition") + tk.MustContainErrMsg(`alter table test_1465 repair partition p1`, "[ddl:8200]Unsupported repair partition") + tk.MustContainErrMsg(`alter table test_1465 import partition p1 tablespace`, "[ddl:8200]Unsupported Unsupported/unknown ALTER TABLE specification") + tk.MustContainErrMsg(`alter table test_1465 discard partition p1 tablespace`, "[ddl:8200]Unsupported Unsupported/unknown ALTER TABLE specification") + tk.MustContainErrMsg(`alter table test_1465 rebuild partition p1`, "[ddl:8200]Unsupported rebuild partition") + tk.MustContainErrMsg(`alter table test_1465 coalesce partition 1`, "[ddl:1509]COALESCE PARTITION can only be used on HASH/KEY partitions") + tk.MustExec("alter table test_1465 partition by hash(a)") + tk.MustQuery(`show create table test_1465`).Check(testkit.Rows("" + + "test_1465 CREATE TABLE `test_1465` (\n" + + " `a` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY HASH (`a`) PARTITIONS 1")) + tk.MustContainErrMsg(`alter table test_1465 drop partition p0`, "[ddl:1512]DROP PARTITION can only be used on RANGE/LIST partitions") } func TestCommitWhenSchemaChange(t *testing.T) { @@ -5183,6 +5248,37 @@ func TestReorgPartitionTiFlash(t *testing.T) { for _, pid := range p.GetAllPartitionIDs() { require.True(t, tbl.Meta().TiFlashReplica.IsPartitionAvailable(pid)) } + tk.MustExec(`alter table t remove partitioning`) + tbl = external.GetTableByName(t, tk, schemaName, "t") + require.Nil(t, tbl.GetPartitionedTable()) + require.NotNil(t, tbl.Meta().TiFlashReplica) + require.True(t, tbl.Meta().TiFlashReplica.Available) + tk.MustExec(`alter table t set tiflash replica 0`) + tbl = external.GetTableByName(t, tk, schemaName, "t") + require.Nil(t, tbl.GetPartitionedTable()) + require.Nil(t, tbl.Meta().TiFlashReplica) + tk.MustExec(`alter table t set tiflash replica 1`) + tbl = external.GetTableByName(t, tk, schemaName, "t") + require.NoError(t, domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tbl.Meta().ID, true)) + tbl = external.GetTableByName(t, tk, schemaName, "t") + require.NotNil(t, tbl.Meta().TiFlashReplica) + require.True(t, tbl.Meta().TiFlashReplica.Available) + require.Nil(t, tbl.GetPartitionedTable()) + tk.MustExec(`alter table t partition by key(a) partitions 3`) + tbl = external.GetTableByName(t, tk, schemaName, "t") + p = tbl.GetPartitionedTable() + for _, pid := range p.GetAllPartitionIDs() { + require.NoError(t, domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), pid, true)) + } + p = tbl.GetPartitionedTable() + require.NotNil(t, tbl.Meta().TiFlashReplica) + require.True(t, tbl.Meta().TiFlashReplica.Available) + for _, pid := range p.GetAllPartitionIDs() { + require.True(t, tbl.Meta().TiFlashReplica.IsPartitionAvailable(pid)) + } + for _, pid := range p.GetAllPartitionIDs() { + require.True(t, tbl.Meta().TiFlashReplica.IsPartitionAvailable(pid)) + } } func TestDuplicatePartitionNames(t *testing.T) { @@ -6042,6 +6138,305 @@ partition p1 values less than maxvalue)`) " PARTITION `p1` VALUES LESS THAN (MAXVALUE))")) } +func TestRemoveRangePartitioning(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database RemovePartitioning") + tk.MustExec("use RemovePartitioning") + tk.MustExec(`create table tRange (a int unsigned primary key, b varchar(255)) +partition by range (a) +(partition p0 values less than (1000000), +partition pMax values less than maxvalue)`) + tk.MustExec(`insert into tRange values (0, "Zero"), (999999, "999999"), (1000000, "1000000"), (20000000, "20000000")`) + tk.MustQuery(`show create table tRange`).Check(testkit.Rows("" + + "tRange CREATE TABLE `tRange` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE (`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (1000000),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + tk.MustExec(`alter table tRange remove partitioning`) + tk.MustQuery(`show create table tRange`).Check(testkit.Rows("" + + "tRange CREATE TABLE `tRange` (\n" + + " `a` int(10) unsigned NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) +} + +func TestRemoveRangeColumnPartitioning(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database RemovePartitioning") + tk.MustExec("use RemovePartitioning") + tk.MustExec(`create table tRange (a varchar(55) primary key, b varchar(255)) +partition by range columns (a) +(partition p0 values less than ("1000000"), +partition pMax values less than maxvalue)`) + tk.MustExec(`insert into tRange values ("0", "Zero"), ("0999999", "0999999"), ("1000000", "1000000"), ("20000000", "20000000")`) + tk.MustQuery(`show create table tRange`).Check(testkit.Rows("" + + "tRange CREATE TABLE `tRange` (\n" + + " `a` varchar(55) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE COLUMNS(`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN ('1000000'),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) + tk.MustExec(`alter table tRange remove partitioning`) + tk.MustQuery(`show create table tRange`).Check(testkit.Rows("" + + "tRange CREATE TABLE `tRange` (\n" + + " `a` varchar(55) NOT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) +} + +func TestRemoveRangeColumnsPartitioning(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database RemovePartitioning") + tk.MustExec("use RemovePartitioning") + tk.MustExec(`create table tRange (a varchar(55), b varchar(255)) +partition by range columns (a,b) +(partition p0 values less than ("1000000","1000000"), +partition pMax values less than (maxvalue,1))`) + tk.MustExec(`insert into tRange values ("0", "0Zero"), ("0999999", "0999999"), ("1000000", "1000000"), ("20000000", "20000000")`) + tk.MustQuery(`show create table tRange`).Check(testkit.Rows("" + + "tRange CREATE TABLE `tRange` (\n" + + " `a` varchar(55) DEFAULT NULL,\n" + + " `b` varchar(255) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE COLUMNS(`a`,`b`)\n" + + "(PARTITION `p0` VALUES LESS THAN ('1000000','1000000'),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE,'1'))")) + tk.MustExec(`alter table tRange remove partitioning`) + tk.MustQuery(`show create table tRange`).Check(testkit.Rows("" + + "tRange CREATE TABLE `tRange` (\n" + + " `a` varchar(55) DEFAULT NULL,\n" + + " `b` varchar(255) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) +} + +func TestRemoveHashPartitioning(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database RemovePartitioning") + tk.MustExec("use RemovePartitioning") + tk.MustExec(`create table t (a int, b varchar(255), key (a,b), key (b)) partition by hash (a) partitions 7`) + for i := 32; i <= 126; i++ { + // Fill the data with int and ascii strings + tk.MustExec(fmt.Sprintf(`insert into t values (%d,char(%d,%d,%d,%d))`, i, i, i, i, i)) + } + tk.MustExec(`analyze table t`) + tk.MustQuery(`select partition_name, table_rows from information_schema.partitions where table_schema = 'RemovePartitioning' and table_name = 't'`).Sort().Check(testkit.Rows(""+ + "p0 14", + "p1 13", + "p2 13", + "p3 13", + "p4 14", + "p5 14", + "p6 14")) + tk.MustExec(`alter table t remove partitioning`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) DEFAULT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " KEY `a` (`a`,`b`),\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) +} + +func TestRemoveKeyPartitioning(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("create database RemovePartitioning") + tk.MustExec("use RemovePartitioning") + tk.MustExec(`create table t (a varchar(255), b varchar(255), key (a,b), key (b)) partition by key (a) partitions 7`) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + // Fill the data with ascii strings + for i := 32; i <= 126; i++ { + tk.MustExec(fmt.Sprintf(`insert into t values (char(%d,%d,%d),char(%d,%d,%d,%d))`, i, i, i, i, i, i, i)) + } + tk.MustExec(`analyze table t`) + tk.MustQuery(`show stats_meta where db_name = 'RemovePartitioning' and table_name = 't'`).Sort().CheckAt([]int{0, 1, 2, 4, 5}, [][]interface{}{ + {"RemovePartitioning", "t", "global", "0", "95"}, + {"RemovePartitioning", "t", "p0", "0", "9"}, + {"RemovePartitioning", "t", "p1", "0", "11"}, + {"RemovePartitioning", "t", "p2", "0", "12"}, + {"RemovePartitioning", "t", "p3", "0", "13"}, + {"RemovePartitioning", "t", "p4", "0", "16"}, + {"RemovePartitioning", "t", "p5", "0", "23"}, + {"RemovePartitioning", "t", "p6", "0", "11"}}) + tk.MustQuery(`select partition_name, table_rows from information_schema.partitions where table_schema = 'RemovePartitioning' and table_name = 't'`).Sort().Check(testkit.Rows(""+ + "p0 9", + "p1 11", + "p2 12", + "p3 13", + "p4 16", + "p5 23", + "p6 11")) + tk.MustExec(`alter table t remove partitioning`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` varchar(255) DEFAULT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " KEY `a` (`a`,`b`),\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + // Statistics are updated asynchronously + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + // And also cached and lazy loaded + h.Clear() + require.NoError(t, h.Update(dom.InfoSchema())) + tk.MustQuery(`show stats_meta where db_name = 'RemovePartitioning' and table_name = 't'`).Sort().CheckAt([]int{0, 1, 2, 4, 5}, [][]interface{}{ + {"RemovePartitioning", "t", "", "0", "95"}}) + tk.MustExec(`analyze table t`) + tk.MustQuery(`show stats_meta where db_name = 'RemovePartitioning' and table_name = 't'`).Sort().CheckAt([]int{0, 1, 2, 4, 5}, [][]interface{}{ + {"RemovePartitioning", "t", "", "0", "95"}}) +} + +func TestRemoveListPartitioning(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("create database RemoveListPartitioning") + tk.MustExec("use RemoveListPartitioning") + tk.MustExec(`create table t (a int, b varchar(255), key (a,b), key (b)) partition by list (a) (partition p0 values in (0), partition p1 values in (1), partition p2 values in (2), partition p3 values in (3), partition p4 values in (4))`) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + // Fill the data with ascii strings + for i := 32; i <= 126; i++ { + tk.MustExec(fmt.Sprintf(`insert into t values (%d,char(%d,%d,%d,%d))`, i%5, i, i, i, i)) + } + tk.MustExec(`analyze table t`) + tk.MustQuery(`show stats_meta where db_name = 'RemoveListPartitioning' and table_name = 't'`).Sort().CheckAt([]int{0, 1, 2, 4, 5}, [][]interface{}{ + {"RemoveListPartitioning", "t", "global", "0", "95"}, + {"RemoveListPartitioning", "t", "p0", "0", "19"}, + {"RemoveListPartitioning", "t", "p1", "0", "19"}, + {"RemoveListPartitioning", "t", "p2", "0", "19"}, + {"RemoveListPartitioning", "t", "p3", "0", "19"}, + {"RemoveListPartitioning", "t", "p4", "0", "19"}}) + tk.MustQuery(`select partition_name, table_rows from information_schema.partitions where table_schema = 'RemoveListPartitioning' and table_name = 't'`).Sort().Check(testkit.Rows(""+ + "p0 19", + "p1 19", + "p2 19", + "p3 19", + "p4 19")) + tk.MustExec(`alter table t remove partitioning`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) DEFAULT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " KEY `a` (`a`,`b`),\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + // Statistics are updated asynchronously + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + // And also cached and lazy loaded + h.Clear() + require.NoError(t, h.Update(dom.InfoSchema())) + tk.MustQuery(`show stats_meta where db_name = 'RemoveListPartitioning' and table_name = 't'`).Sort().CheckAt([]int{0, 1, 2, 4, 5}, [][]interface{}{ + {"RemoveListPartitioning", "t", "", "0", "95"}}) + tk.MustExec(`analyze table t`) + tk.MustQuery(`show stats_meta where db_name = 'RemoveListPartitioning' and table_name = 't'`).Sort().CheckAt([]int{0, 1, 2, 4, 5}, [][]interface{}{ + {"RemoveListPartitioning", "t", "", "0", "95"}}) +} + +func TestRemoveListColumnPartitioning(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("create database RemoveListPartitioning") + tk.MustExec("use RemoveListPartitioning") + tk.MustExec(`create table t (a varchar(255), b varchar(255), key (a,b), key (b)) partition by list columns (a) (partition p0 values in ("0"), partition p1 values in ("1"), partition p2 values in ("2"), partition p3 values in ("3"), partition p4 values in ("4"))`) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + // Fill the data with ascii strings + for i := 32; i <= 126; i++ { + tk.MustExec(fmt.Sprintf(`insert into t values ("%d",char(%d,%d,%d,%d))`, i%5, i, i, i, i)) + } + tk.MustExec(`analyze table t`) + tk.MustQuery(`show stats_meta where db_name = 'RemoveListPartitioning' and table_name = 't'`).Sort().CheckAt([]int{0, 1, 2, 4, 5}, [][]interface{}{ + {"RemoveListPartitioning", "t", "global", "0", "95"}, + {"RemoveListPartitioning", "t", "p0", "0", "19"}, + {"RemoveListPartitioning", "t", "p1", "0", "19"}, + {"RemoveListPartitioning", "t", "p2", "0", "19"}, + {"RemoveListPartitioning", "t", "p3", "0", "19"}, + {"RemoveListPartitioning", "t", "p4", "0", "19"}}) + tk.MustQuery(`select partition_name, table_rows from information_schema.partitions where table_schema = 'RemoveListPartitioning' and table_name = 't'`).Sort().Check(testkit.Rows(""+ + "p0 19", + "p1 19", + "p2 19", + "p3 19", + "p4 19")) + tk.MustExec(`alter table t remove partitioning`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` varchar(255) DEFAULT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " KEY `a` (`a`,`b`),\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + // Statistics are updated asynchronously + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + // And also cached and lazy loaded + h.Clear() + require.NoError(t, h.Update(dom.InfoSchema())) + tk.MustQuery(`show stats_meta where db_name = 'RemoveListPartitioning' and table_name = 't'`).Sort().CheckAt([]int{0, 1, 2, 4, 5}, [][]interface{}{ + {"RemoveListPartitioning", "t", "", "0", "95"}}) + tk.MustExec(`analyze table t`) + tk.MustQuery(`show stats_meta where db_name = 'RemoveListPartitioning' and table_name = 't'`).Sort().CheckAt([]int{0, 1, 2, 4, 5}, [][]interface{}{ + {"RemoveListPartitioning", "t", "", "0", "95"}}) +} + +func TestRemoveListColumnsPartitioning(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + tk.MustExec("create database RemoveListPartitioning") + tk.MustExec("use RemoveListPartitioning") + tk.MustExec(`create table t (a int, b varchar(255), key (a,b), key (b)) partition by list columns (a,b) (partition p0 values in ((0,"0")), partition p1 values in ((1,"1")), partition p2 values in ((2,"2")), partition p3 values in ((3,"3")), partition p4 values in ((4,"4")))`) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + // Fill the data + for i := 32; i <= 126; i++ { + tk.MustExec(fmt.Sprintf(`insert into t values (%d,"%d")`, i%5, i%5)) + } + tk.MustExec(`analyze table t`) + tk.MustQuery(`show stats_meta where db_name = 'RemoveListPartitioning' and table_name = 't'`).Sort().CheckAt([]int{0, 1, 2, 4, 5}, [][]interface{}{ + {"RemoveListPartitioning", "t", "global", "0", "95"}, + {"RemoveListPartitioning", "t", "p0", "0", "19"}, + {"RemoveListPartitioning", "t", "p1", "0", "19"}, + {"RemoveListPartitioning", "t", "p2", "0", "19"}, + {"RemoveListPartitioning", "t", "p3", "0", "19"}, + {"RemoveListPartitioning", "t", "p4", "0", "19"}}) + tk.MustQuery(`select partition_name, table_rows from information_schema.partitions where table_schema = 'RemoveListPartitioning' and table_name = 't'`).Sort().Check(testkit.Rows(""+ + "p0 19", + "p1 19", + "p2 19", + "p3 19", + "p4 19")) + tk.MustExec(`alter table t remove partitioning`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("" + + "t CREATE TABLE `t` (\n" + + " `a` int(11) DEFAULT NULL,\n" + + " `b` varchar(255) DEFAULT NULL,\n" + + " KEY `a` (`a`,`b`),\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + // Statistics are updated asynchronously + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + // And also cached and lazy loaded + h.Clear() + require.NoError(t, h.Update(dom.InfoSchema())) + tk.MustQuery(`show stats_meta where db_name = 'RemoveListPartitioning' and table_name = 't'`).Sort().CheckAt([]int{0, 1, 2, 4, 5}, [][]interface{}{ + {"RemoveListPartitioning", "t", "", "0", "95"}}) + tk.MustExec(`analyze table t`) + tk.MustQuery(`show stats_meta where db_name = 'RemoveListPartitioning' and table_name = 't'`).Sort().CheckAt([]int{0, 1, 2, 4, 5}, [][]interface{}{ + {"RemoveListPartitioning", "t", "", "0", "95"}}) +} + func TestListDefinitionError(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index efe547bfe7da1..ec458bcd52bce 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -229,7 +229,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { if len(e.planPhysIDs) > 0 { physID = e.planPhysIDs[i] } else { - physID, err = core.GetPhysID(e.tblInfo, e.partExpr, idxVals[e.partPos]) + physID, err = core.GetPhysID(e.tblInfo, e.partExpr, e.partPos, idxVals[e.partPos]) if err != nil { continue } @@ -364,7 +364,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { } else { if handle.IsInt() { d := types.NewIntDatum(handle.IntValue()) - tID, err = core.GetPhysID(e.tblInfo, e.partExpr, d) + tID, err = core.GetPhysID(e.tblInfo, e.partExpr, e.partPos, d) if err != nil { continue } @@ -373,7 +373,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { if err1 != nil { return err1 } - tID, err = core.GetPhysID(e.tblInfo, e.partExpr, d) + tID, err = core.GetPhysID(e.tblInfo, e.partExpr, e.partPos, d) if err != nil { continue } diff --git a/expression/simple_rewriter.go b/expression/simple_rewriter.go index 913c0970a8bcd..c8c586d5891be 100644 --- a/expression/simple_rewriter.go +++ b/expression/simple_rewriter.go @@ -29,6 +29,9 @@ import ( // ParseSimpleExprWithTableInfo parses simple expression string to Expression. // The expression string must only reference the column in table Info. func ParseSimpleExprWithTableInfo(ctx sessionctx.Context, exprStr string, tableInfo *model.TableInfo) (Expression, error) { + if len(exprStr) == 0 { + return nil, nil + } exprStr = "select " + exprStr var stmts []ast.StmtNode var err error diff --git a/infoschema/builder.go b/infoschema/builder.go index 6008a30d21f65..ee04d84686cae 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -226,7 +226,8 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro return b.applyRecoverTable(m, diff) case model.ActionCreateTables: return b.applyCreateTables(m, diff) - case model.ActionReorganizePartition: + case model.ActionReorganizePartition, model.ActionRemovePartitioning, + model.ActionAlterTablePartitioning: return b.applyReorganizePartition(m, diff) case model.ActionExchangeTablePartition: return b.applyExchangeTablePartition(m, diff) @@ -308,6 +309,7 @@ func (b *Builder) applyReorganizePartition(m *meta.Meta, diff *model.SchemaDiff) if opt.TableID != 0 { b.markTableBundleShouldUpdate(opt.TableID) } + // TODO: Should we also check markPartitionBundleShouldUpdate?!? } return tblIDs, nil } @@ -467,7 +469,8 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6 case model.ActionDropTable, model.ActionDropView, model.ActionDropSequence: oldTableID = diff.TableID case model.ActionTruncateTable, model.ActionCreateView, - model.ActionExchangeTablePartition: + model.ActionExchangeTablePartition, model.ActionAlterTablePartitioning, + model.ActionRemovePartitioning: oldTableID = diff.OldTableID newTableID = diff.TableID default: @@ -768,7 +771,8 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i case model.ActionDropTablePartition: case model.ActionTruncateTablePartition: // ReorganizePartition handle the bundles in applyReorganizePartition - case model.ActionReorganizePartition: + case model.ActionReorganizePartition, model.ActionRemovePartitioning, + model.ActionAlterTablePartitioning: default: pi := tblInfo.GetPartitionInfo() if pi != nil { diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index b1a0f9ab7fee6..ec184f5adbc6f 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -657,6 +657,8 @@ func NewSequenceAllocator(store kv.Storage, dbID, tbID int64, info *model.Sequen } } +// TODO: Handle allocators when changing Table ID during ALTER TABLE t PARTITION BY ... + // NewAllocatorsFromTblInfo creates an array of allocators of different types with the information of model.TableInfo. func NewAllocatorsFromTblInfo(store kv.Storage, schemaID int64, tblInfo *model.TableInfo) Allocators { var allocs []Allocator diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 9a0043a6da144..3ddbbde13a183 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -102,6 +102,8 @@ const ( ActionCreateResourceGroup ActionType = 68 ActionAlterResourceGroup ActionType = 69 ActionDropResourceGroup ActionType = 70 + ActionAlterTablePartitioning ActionType = 71 + ActionRemovePartitioning ActionType = 72 ) var actionMap = map[ActionType]string{ @@ -170,6 +172,8 @@ var actionMap = map[ActionType]string{ ActionCreateResourceGroup: "create resource group", ActionAlterResourceGroup: "alter resource group", ActionDropResourceGroup: "drop resource group", + ActionAlterTablePartitioning: "alter table partition by", + ActionRemovePartitioning: "alter table remove partitioning", // `ActionAlterTableAlterPartition` is removed and will never be used. // Just left a tombstone here for compatibility. @@ -797,7 +801,8 @@ func (job *Job) NotStarted() bool { // MayNeedReorg indicates that this job may need to reorganize the data. func (job *Job) MayNeedReorg() bool { switch job.Type { - case ActionAddIndex, ActionAddPrimaryKey, ActionReorganizePartition: + case ActionAddIndex, ActionAddPrimaryKey, ActionReorganizePartition, + ActionRemovePartitioning, ActionAlterTablePartitioning: return true case ActionModifyColumn: if len(job.CtxVars) > 0 { diff --git a/parser/model/ddl_test.go b/parser/model/ddl_test.go index c5d3d5ebb25b2..6f7d0a4bbe5e5 100644 --- a/parser/model/ddl_test.go +++ b/parser/model/ddl_test.go @@ -76,6 +76,8 @@ func TestMayNeedReorg(t *testing.T) { //TODO(bb7133): add more test cases for different ActionType. reorgJobTypes := []model.ActionType{ model.ActionReorganizePartition, + model.ActionRemovePartitioning, + model.ActionAlterTablePartitioning, model.ActionAddIndex, model.ActionAddPrimaryKey, } diff --git a/parser/model/model.go b/parser/model/model.go index 30dfaa4127408..a0889e04dea94 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -1132,6 +1132,10 @@ type PartitionType int // Partition types. const ( + // Actually non-partitioned, but during DDL keeping the table as + // a single partition + PartitionTypeNone PartitionType = 0 + PartitionTypeRange PartitionType = 1 PartitionTypeHash PartitionType = 2 PartitionTypeList PartitionType = 3 @@ -1151,6 +1155,8 @@ func (p PartitionType) String() string { return "KEY" case PartitionTypeSystemTime: return "SYSTEM_TIME" + case PartitionTypeNone: + return "NONE" default: return "" } @@ -1187,6 +1193,16 @@ type PartitionInfo struct { Num uint64 `json:"num"` // Only used during ReorganizePartition so far DDLState SchemaState `json:"ddl_state"` + // Set during ALTER TABLE ... if the table id needs to change + // like if there is a global index or going between non-partitioned + // and partitioned table, to make the data dropping / range delete + // optimized. + NewTableID int64 `json:"new_table_id"` + // Set during ALTER TABLE ... PARTITION BY ... + // First as the new partition scheme, then in StateDeleteReorg as the old + DDLType PartitionType `json:"ddl_type"` + DDLExpr string `json:"ddl_expr"` + DDLColumns []CIStr `json:"ddl_columns"` } // Clone clones itself. diff --git a/parser/parser_test.go b/parser/parser_test.go index e7d88aad49a3d..94e5864ddfefa 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -2709,6 +2709,9 @@ func TestDDL(t *testing.T) { {`alter table m add partition (partition p1 values less than (200) learner_constraints="ww");`, false, ""}, {`alter table m add partition (partition p1 values less than (200) placement policy="ww");`, true, "ALTER TABLE `m` ADD PARTITION (PARTITION `p1` VALUES LESS THAN (200) PLACEMENT POLICY = `ww`)"}, {`alter table m add partition (partition p1 values less than (200) /*T![placement] placement policy="ww" */);`, true, "ALTER TABLE `m` ADD PARTITION (PARTITION `p1` VALUES LESS THAN (200) PLACEMENT POLICY = `ww`)"}, + {`alter table m add column a int, add partition (partition p1 values less than (200))`, true, "ALTER TABLE `m` ADD COLUMN `a` INT, ADD PARTITION (PARTITION `p1` VALUES LESS THAN (200))"}, + // TODO: Do not allow this order! + {`alter table m add partition (partition p1 values less than (200)), add column a int`, true, "ALTER TABLE `m` ADD PARTITION (PARTITION `p1` VALUES LESS THAN (200)), ADD COLUMN `a` INT"}, // for check clause {"create table t (c1 bool, c2 bool, check (c1 in (0, 1)) not enforced, check (c2 in (0, 1)))", true, "CREATE TABLE `t` (`c1` TINYINT(1),`c2` TINYINT(1),CHECK(`c1` IN (0,1)) NOT ENFORCED,CHECK(`c2` IN (0,1)) ENFORCED)"}, {"CREATE TABLE Customer (SD integer CHECK (SD > 0), First_Name varchar(30));", true, "CREATE TABLE `Customer` (`SD` INT CHECK(`SD`>0) ENFORCED,`First_Name` VARCHAR(30))"}, @@ -3166,11 +3169,15 @@ func TestDDL(t *testing.T) { {"alter table t analyze partition a index b with 4 buckets", true, "ANALYZE TABLE `t` PARTITION `a` INDEX `b` WITH 4 BUCKETS"}, {"alter table t partition by hash(a)", true, "ALTER TABLE `t` PARTITION BY HASH (`a`) PARTITIONS 1"}, + {"alter table t add column a int partition by hash(a)", true, "ALTER TABLE `t` ADD COLUMN `a` INT PARTITION BY HASH (`a`) PARTITIONS 1"}, {"alter table t partition by range(a)", false, ""}, {"alter table t partition by range(a) (partition x values less than (75))", true, "ALTER TABLE `t` PARTITION BY RANGE (`a`) (PARTITION `x` VALUES LESS THAN (75))"}, + {"alter table t add column a int, partition by range(a) (partition x values less than (75))", false, ""}, {"alter table t comment 'cmt' partition by hash(a)", true, "ALTER TABLE `t` COMMENT = 'cmt' PARTITION BY HASH (`a`) PARTITIONS 1"}, {"alter table t enable keys, comment = 'cmt' partition by hash(a)", true, "ALTER TABLE `t` ENABLE KEYS, COMMENT = 'cmt' PARTITION BY HASH (`a`) PARTITIONS 1"}, {"alter table t enable keys, comment = 'cmt', partition by hash(a)", false, ""}, + {"alter table t partition by hash(a) enable keys", false, ""}, + {"alter table t partition by hash(a), enable keys", false, ""}, // Test keyword `FIELDS` {"alter table t partition by range FIELDS(a) (partition x values less than maxvalue)", true, "ALTER TABLE `t` PARTITION BY RANGE COLUMNS (`a`) (PARTITION `x` VALUES LESS THAN (MAXVALUE))"}, @@ -3385,6 +3392,12 @@ func TestDDL(t *testing.T) { {"alter table t remove partitioning", true, "ALTER TABLE `t` REMOVE PARTITIONING"}, {"alter table db.ident remove partitioning", true, "ALTER TABLE `db`.`ident` REMOVE PARTITIONING"}, {"alter table t lock = default remove partitioning", true, "ALTER TABLE `t` LOCK = DEFAULT REMOVE PARTITIONING"}, + {"alter table t add column a int remove partitioning", true, "ALTER TABLE `t` ADD COLUMN `a` INT REMOVE PARTITIONING"}, + {"alter table t add column a int, add index (c) remove partitioning", true, "ALTER TABLE `t` ADD COLUMN `a` INT, ADD INDEX(`c`) REMOVE PARTITIONING"}, + {"alter table t add column a int, remove partitioning", false, ""}, + {"alter table t add column a int, add index (c), remove partitioning", false, ""}, + {"alter table t remove partitioning add column a int", false, ""}, + {"alter table t remove partitioning, add column a int", false, ""}, // for references without IndexColNameList {"alter table t add column a double (4,2) zerofill references b match full on update set null first", true, "ALTER TABLE `t` ADD COLUMN `a` DOUBLE(4,2) UNSIGNED ZEROFILL REFERENCES `b` MATCH FULL ON UPDATE SET NULL FIRST"}, diff --git a/planner/core/initialize.go b/planner/core/initialize.go index 8d4d88d47aa96..ebcc1be1ef2fd 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -569,7 +569,7 @@ func (p *BatchPointGetPlan) Init(ctx sessionctx.Context, stats *property.StatsIn break } } - pid, err := GetPhysID(p.TblInfo, p.PartitionExpr, d) + pid, err := GetPhysID(p.TblInfo, p.PartitionExpr, p.PartitionColPos, d) if err != nil { hasErr = true break @@ -578,7 +578,7 @@ func (p *BatchPointGetPlan) Init(ctx sessionctx.Context, stats *property.StatsIn } } else { for _, idxVals := range p.IndexValues { - pid, err := GetPhysID(p.TblInfo, p.PartitionExpr, idxVals[p.PartitionColPos]) + pid, err := GetPhysID(p.TblInfo, p.PartitionExpr, p.PartitionColPos, idxVals[p.PartitionColPos]) if err != nil { hasErr = true break diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 0e388d3cbea7b..5a013ae6e7edc 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -1821,7 +1821,7 @@ func getPartitionInfo(ctx sessionctx.Context, tbl *model.TableInfo, pairs []name if len(pi.Columns) == 1 { for i, pair := range pairs { if pi.Columns[0].L == pair.colName { - pos, err := partitionExpr.LocateKeyPartitionWithSPC(pi, []types.Datum{pair.value}) + pos, err := partitionExpr.LocateKeyPartition(pi.Num, []types.Datum{pair.value}) if err != nil { return nil, 0, 0, false } diff --git a/planner/core/util.go b/planner/core/util.go index 48a12f7cdc04c..d456fe10374bf 100644 --- a/planner/core/util.go +++ b/planner/core/util.go @@ -414,7 +414,7 @@ func clonePhysicalPlan(plans []PhysicalPlan) ([]PhysicalPlan, error) { } // GetPhysID returns the physical table ID. -func GetPhysID(tblInfo *model.TableInfo, partitionExpr *tables.PartitionExpr, d types.Datum) (int64, error) { +func GetPhysID(tblInfo *model.TableInfo, partitionExpr *tables.PartitionExpr, colPos int, d types.Datum) (int64, error) { pi := tblInfo.GetPartitionInfo() if pi == nil { return tblInfo.ID, nil @@ -430,10 +430,17 @@ func GetPhysID(tblInfo *model.TableInfo, partitionExpr *tables.PartitionExpr, d partIdx := mathutil.Abs(intVal % int64(pi.Num)) return pi.Definitions[partIdx].ID, nil case model.PartitionTypeKey: - if len(pi.Columns) > 1 { + if partitionExpr.ForKeyPruning == nil || + len(pi.Columns) > 1 { return 0, errors.Errorf("unsupported partition type in BatchGet") } - partIdx, err := partitionExpr.LocateKeyPartitionWithSPC(pi, []types.Datum{d}) + newKeyPartExpr := tables.ForKeyPruning{ + KeyPartCols: []*expression.Column{{ + Index: colPos, + UniqueID: partitionExpr.KeyPartCols[0].UniqueID, + }}, + } + partIdx, err := newKeyPartExpr.LocateKeyPartition(pi.Num, []types.Datum{d}) if err != nil { return 0, errors.Errorf("unsupported partition type in BatchGet") } diff --git a/statistics/handle/ddl.go b/statistics/handle/ddl.go index 2688ab18364e6..ed3cc0c69c877 100644 --- a/statistics/handle/ddl.go +++ b/statistics/handle/ddl.go @@ -69,8 +69,23 @@ func (h *Handle) HandleDDLEvent(t *util.Event) error { if err := h.insertTableStats2KV(t.TableInfo, def.ID); err != nil { return err } + // Do not update global stats, since the data have not changed! } - // Do not update global stats, since the data have not changed! + case model.ActionAlterTablePartitioning: + // Add partitioning + for _, def := range t.PartInfo.Definitions { + // TODO: Should we trigger analyze instead of adding 0s? + if err := h.insertTableStats2KV(t.TableInfo, def.ID); err != nil { + return err + } + } + fallthrough + case model.ActionRemovePartitioning: + // Change id for global stats, since the data has not changed! + // Note that t.TableInfo is the current (new) table info + // and t.PartInfo.NewTableID is actually the old table ID! + // (see onReorganizePartition) + return h.changeGlobalStatsID(t.PartInfo.NewTableID, t.TableInfo.ID) case model.ActionFlashbackCluster: return h.updateStatsVersion() } @@ -222,6 +237,27 @@ func (h *Handle) updateGlobalStats(tblInfo *model.TableInfo) error { return nil } +func (h *Handle) changeGlobalStatsID(from, to int64) (err error) { + h.mu.Lock() + defer h.mu.Unlock() + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) + exec := h.mu.ctx.(sqlexec.SQLExecutor) + _, err = exec.ExecuteInternal(ctx, "begin pessimistic") + if err != nil { + return errors.Trace(err) + } + defer func() { + err = finishTransaction(ctx, exec, err) + }() + for _, table := range []string{"stats_meta", "stats_top_n", "stats_fm_sketch", "stats_buckets", "stats_histograms", "column_stats_usage"} { + _, err = exec.ExecuteInternal(ctx, "update mysql."+table+" set table_id = %? where table_id = %?", to, from) + if err != nil { + return err + } + } + return nil +} + func (h *Handle) getInitStateTableIDs(tblInfo *model.TableInfo) (ids []int64) { pi := tblInfo.GetPartitionInfo() if pi == nil { diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index 6140f6c08d1e3..3640887801dad 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -2141,11 +2141,9 @@ func (w *GCWorker) doGCPlacementRules(se session.Session, safePoint uint64, dr u return } physicalTableIDs = append(physicalTableIDs, historyJob.TableID) - case model.ActionDropSchema, model.ActionDropTablePartition, model.ActionTruncateTablePartition: - if err = historyJob.DecodeArgs(&physicalTableIDs); err != nil { - return - } - case model.ActionReorganizePartition: + case model.ActionDropSchema, model.ActionDropTablePartition, model.ActionTruncateTablePartition, + model.ActionReorganizePartition, model.ActionRemovePartitioning, + model.ActionAlterTablePartitioning: if err = historyJob.DecodeArgs(&physicalTableIDs); err != nil { return } diff --git a/table/tables/BUILD.bazel b/table/tables/BUILD.bazel index 46c181cdff385..682919b06418e 100644 --- a/table/tables/BUILD.bazel +++ b/table/tables/BUILD.bazel @@ -61,7 +61,7 @@ go_library( go_test( name = "tables_test", - timeout = "short", + timeout = "long", srcs = [ "cache_test.go", "export_test.go", diff --git a/table/tables/partition.go b/table/tables/partition.go index 60f4aad6dbb7f..7f55d61544cd1 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -112,7 +112,7 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part return nil, table.ErrUnknownPartition } ret := &partitionedTable{TableCommon: *tbl} - partitionExpr, err := newPartitionExpr(tblInfo, pi.Definitions) + partitionExpr, err := newPartitionExpr(tblInfo, pi.Type, pi.Expr, pi.Columns, pi.Definitions) if err != nil { return nil, errors.Trace(err) } @@ -147,7 +147,12 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part if pi.DDLState == model.StateDeleteReorganization { origIdx := setIndexesState(ret, pi.DDLState) defer unsetIndexesState(ret, origIdx) - ret.reorgPartitionExpr, err = newPartitionExpr(tblInfo, pi.DroppingDefinitions) + // TODO: Explicitly explain the different DDL/New fields! + if pi.NewTableID != 0 { + ret.reorgPartitionExpr, err = newPartitionExpr(tblInfo, pi.DDLType, pi.DDLExpr, pi.DDLColumns, pi.DroppingDefinitions) + } else { + ret.reorgPartitionExpr, err = newPartitionExpr(tblInfo, pi.Type, pi.Expr, pi.Columns, pi.DroppingDefinitions) + } if err != nil { return nil, errors.Trace(err) } @@ -168,7 +173,13 @@ func newPartitionedTable(tbl *TableCommon, tblInfo *model.TableInfo) (table.Part if len(pi.AddingDefinitions) > 0 { origIdx := setIndexesState(ret, pi.DDLState) defer unsetIndexesState(ret, origIdx) - ret.reorgPartitionExpr, err = newPartitionExpr(tblInfo, pi.AddingDefinitions) + if pi.NewTableID != 0 { + // REMOVE PARTITIONING or PARTITION BY + ret.reorgPartitionExpr, err = newPartitionExpr(tblInfo, pi.DDLType, pi.DDLExpr, pi.DDLColumns, pi.AddingDefinitions) + } else { + // REORGANIZE PARTITION + ret.reorgPartitionExpr, err = newPartitionExpr(tblInfo, pi.Type, pi.Expr, pi.Columns, pi.AddingDefinitions) + } if err != nil { return nil, errors.Trace(err) } @@ -227,7 +238,7 @@ func initPartition(t *partitionedTable, def model.PartitionDefinition) (*partiti return &newPart, nil } -func newPartitionExpr(tblInfo *model.TableInfo, defs []model.PartitionDefinition) (*PartitionExpr, error) { +func newPartitionExpr(tblInfo *model.TableInfo, tp model.PartitionType, expr string, partCols []model.CIStr, defs []model.PartitionDefinition) (*PartitionExpr, error) { // a partitioned table cannot rely on session context/sql modes, so use a default one! ctx := mock.NewContext() dbName := model.NewCIStr(ctx.GetSessionVars().CurrentDB) @@ -235,16 +246,18 @@ func newPartitionExpr(tblInfo *model.TableInfo, defs []model.PartitionDefinition if err != nil { return nil, err } - pi := tblInfo.GetPartitionInfo() - switch pi.Type { + switch tp { + case model.PartitionTypeNone: + // Nothing to do + return nil, nil case model.PartitionTypeRange: - return generateRangePartitionExpr(ctx, pi, defs, columns, names) + return generateRangePartitionExpr(ctx, expr, partCols, defs, columns, names) case model.PartitionTypeHash: - return generateHashPartitionExpr(ctx, pi, columns, names) + return generateHashPartitionExpr(ctx, expr, columns, names) case model.PartitionTypeKey: - return generateKeyPartitionExpr(ctx, pi, columns, names) + return generateKeyPartitionExpr(ctx, expr, partCols, columns, names) case model.PartitionTypeList: - return generateListPartitionExpr(ctx, tblInfo, defs, columns, names) + return generateListPartitionExpr(ctx, tblInfo, expr, partCols, defs, columns, names) } panic("cannot reach here") } @@ -281,17 +294,6 @@ func (pe *PartitionExpr) GetPartColumnsForKeyPartition(columns []*expression.Col return partCols, colLen } -// LocateKeyPartitionWithSPC is used to locate the destination partition for key -// partition table has single partition column(SPC). It's called in FastPlan process. -func (pe *PartitionExpr) LocateKeyPartitionWithSPC(pi *model.PartitionInfo, - r []types.Datum) (int, error) { - col := &expression.Column{} - *col = *pe.KeyPartCols[0] - col.Index = 0 - kp := &ForKeyPruning{KeyPartCols: []*expression.Column{col}} - return kp.LocateKeyPartition(pi.Num, r) -} - // LocateKeyPartition is the common interface used to locate the destination partition func (kp *ForKeyPruning) LocateKeyPartition(numParts uint64, r []types.Datum) (int, error) { h := crc32.NewIEEE() @@ -654,25 +656,25 @@ func fixOldVersionPartitionInfo(sctx sessionctx.Context, str string) (int64, boo return ret, true } -func rangePartitionExprStrings(pi *model.PartitionInfo) []string { +func rangePartitionExprStrings(cols []model.CIStr, expr string) []string { var s []string - if len(pi.Columns) > 0 { - s = make([]string, 0, len(pi.Columns)) - for _, col := range pi.Columns { + if len(cols) > 0 { + s = make([]string, 0, len(cols)) + for _, col := range cols { s = append(s, stringutil.Escape(col.O, mysql.ModeNone)) } } else { - s = []string{pi.Expr} + s = []string{expr} } return s } -func generateKeyPartitionExpr(ctx sessionctx.Context, pi *model.PartitionInfo, +func generateKeyPartitionExpr(ctx sessionctx.Context, expr string, partCols []model.CIStr, columns []*expression.Column, names types.NameSlice) (*PartitionExpr, error) { ret := &PartitionExpr{ ForKeyPruning: &ForKeyPruning{}, } - _, partColumns, offset, err := extractPartitionExprColumns(ctx, pi, columns, names) + _, partColumns, offset, err := extractPartitionExprColumns(ctx, expr, partCols, columns, names) if err != nil { return nil, errors.Trace(err) } @@ -682,12 +684,12 @@ func generateKeyPartitionExpr(ctx sessionctx.Context, pi *model.PartitionInfo, return ret, nil } -func generateRangePartitionExpr(ctx sessionctx.Context, pi *model.PartitionInfo, +func generateRangePartitionExpr(ctx sessionctx.Context, expr string, partCols []model.CIStr, defs []model.PartitionDefinition, columns []*expression.Column, names types.NameSlice) (*PartitionExpr, error) { // The caller should assure partition info is not nil. p := parser.New() schema := expression.NewSchema(columns...) - partStrs := rangePartitionExprStrings(pi) + partStrs := rangePartitionExprStrings(partCols, expr) locateExprs, err := getRangeLocateExprs(ctx, p, defs, partStrs, schema, names) if err != nil { return nil, errors.Trace(err) @@ -696,13 +698,13 @@ func generateRangePartitionExpr(ctx sessionctx.Context, pi *model.PartitionInfo, UpperBounds: locateExprs, } - partExpr, _, offset, err := extractPartitionExprColumns(ctx, pi, columns, names) + partExpr, _, offset, err := extractPartitionExprColumns(ctx, expr, partCols, columns, names) if err != nil { return nil, errors.Trace(err) } ret.ColumnOffset = offset - if len(pi.Columns) < 1 { + if len(partCols) < 1 { tmp, err := dataForRangePruning(ctx, defs) if err != nil { return nil, errors.Trace(err) @@ -772,19 +774,19 @@ func findIdxByColUniqueID(cols []*expression.Column, col *expression.Column) int return -1 } -func extractPartitionExprColumns(ctx sessionctx.Context, pi *model.PartitionInfo, columns []*expression.Column, names types.NameSlice) (expression.Expression, []*expression.Column, []int, error) { +func extractPartitionExprColumns(ctx sessionctx.Context, expr string, partCols []model.CIStr, columns []*expression.Column, names types.NameSlice) (expression.Expression, []*expression.Column, []int, error) { var cols []*expression.Column var partExpr expression.Expression - if len(pi.Columns) == 0 { + if len(partCols) == 0 { schema := expression.NewSchema(columns...) - exprs, err := expression.ParseSimpleExprsWithNames(ctx, pi.Expr, schema, names) + exprs, err := expression.ParseSimpleExprsWithNames(ctx, expr, schema, names) if err != nil { return nil, nil, nil, err } cols = expression.ExtractColumns(exprs[0]) partExpr = exprs[0] } else { - for _, col := range pi.Columns { + for _, col := range partCols { idx := expression.FindFieldNameIdxByColName(names, col.L) if idx < 0 { panic("should never happen") @@ -803,19 +805,18 @@ func extractPartitionExprColumns(ctx sessionctx.Context, pi *model.PartitionInfo return partExpr, deDupCols, offset, nil } -func generateListPartitionExpr(ctx sessionctx.Context, tblInfo *model.TableInfo, +func generateListPartitionExpr(ctx sessionctx.Context, tblInfo *model.TableInfo, expr string, partCols []model.CIStr, defs []model.PartitionDefinition, columns []*expression.Column, names types.NameSlice) (*PartitionExpr, error) { // The caller should assure partition info is not nil. - pi := tblInfo.GetPartitionInfo() - partExpr, exprCols, offset, err := extractPartitionExprColumns(ctx, pi, columns, names) + partExpr, exprCols, offset, err := extractPartitionExprColumns(ctx, expr, partCols, columns, names) if err != nil { return nil, err } listPrune := &ForListPruning{} - if len(pi.Columns) == 0 { - err = listPrune.buildListPruner(ctx, tblInfo, defs, exprCols, columns, names) + if len(partCols) == 0 { + err = listPrune.buildListPruner(ctx, expr, defs, exprCols, columns, names) } else { - err = listPrune.buildListColumnsPruner(ctx, tblInfo, defs, columns, names) + err = listPrune.buildListColumnsPruner(ctx, tblInfo, partCols, defs, columns, names) } if err != nil { return nil, err @@ -851,15 +852,14 @@ func (lp *ForListPruning) Clone() *ForListPruning { return &ret } -func (lp *ForListPruning) buildListPruner(ctx sessionctx.Context, tblInfo *model.TableInfo, defs []model.PartitionDefinition, exprCols []*expression.Column, +func (lp *ForListPruning) buildListPruner(ctx sessionctx.Context, exprStr string, defs []model.PartitionDefinition, exprCols []*expression.Column, columns []*expression.Column, names types.NameSlice) error { - pi := tblInfo.GetPartitionInfo() schema := expression.NewSchema(columns...) p := parser.New() - expr, err := parseSimpleExprWithNames(p, ctx, pi.Expr, schema, names) + expr, err := parseSimpleExprWithNames(p, ctx, exprStr, schema, names) if err != nil { // If it got an error here, ddl may hang forever, so this error log is important. - logutil.BgLogger().Error("wrong table partition expression", zap.String("expression", pi.Expr), zap.Error(err)) + logutil.BgLogger().Error("wrong table partition expression", zap.String("expression", exprStr), zap.Error(err)) return errors.Trace(err) } // Since need to change the column index of the expression, clone the expression first. @@ -882,21 +882,20 @@ func (lp *ForListPruning) buildListPruner(ctx sessionctx.Context, tblInfo *model } func (lp *ForListPruning) buildListColumnsPruner(ctx sessionctx.Context, - tblInfo *model.TableInfo, defs []model.PartitionDefinition, + tblInfo *model.TableInfo, partCols []model.CIStr, defs []model.PartitionDefinition, columns []*expression.Column, names types.NameSlice) error { - pi := tblInfo.GetPartitionInfo() schema := expression.NewSchema(columns...) p := parser.New() + colPrunes := make([]*ForListColumnPruning, 0, len(partCols)) lp.defaultPartitionIdx = -1 - colPrunes := make([]*ForListColumnPruning, 0, len(pi.Columns)) - for colIdx := range pi.Columns { - colInfo := model.FindColumnInfo(tblInfo.Columns, pi.Columns[colIdx].L) + for colIdx := range partCols { + colInfo := model.FindColumnInfo(tblInfo.Columns, partCols[colIdx].L) if colInfo == nil { - return table.ErrUnknownColumn.GenWithStackByArgs(pi.Columns[colIdx].L) + return table.ErrUnknownColumn.GenWithStackByArgs(partCols[colIdx].L) } - idx := expression.FindFieldNameIdxByColName(names, pi.Columns[colIdx].L) + idx := expression.FindFieldNameIdxByColName(names, partCols[colIdx].L) if idx < 0 { - return table.ErrUnknownColumn.GenWithStackByArgs(pi.Columns[colIdx].L) + return table.ErrUnknownColumn.GenWithStackByArgs(partCols[colIdx].L) } colPrune := &ForListColumnPruning{ ctx: ctx, @@ -1182,18 +1181,18 @@ func (lp *ForListColumnPruning) LocateRanges(sc *stmtctx.StatementContext, r *ra return locations, nil } -func generateHashPartitionExpr(ctx sessionctx.Context, pi *model.PartitionInfo, +func generateHashPartitionExpr(ctx sessionctx.Context, exprStr string, columns []*expression.Column, names types.NameSlice) (*PartitionExpr, error) { // The caller should assure partition info is not nil. schema := expression.NewSchema(columns...) - origExpr, err := parseExpr(parser.New(), pi.Expr) + origExpr, err := parseExpr(parser.New(), exprStr) if err != nil { return nil, err } exprs, err := rewritePartitionExpr(ctx, origExpr, schema, names) if err != nil { // If it got an error here, ddl may hang forever, so this error log is important. - logutil.BgLogger().Error("wrong table partition expression", zap.String("expression", pi.Expr), zap.Error(err)) + logutil.BgLogger().Error("wrong table partition expression", zap.String("expression", exprStr), zap.Error(err)) return nil, errors.Trace(err) } // build column offset. @@ -1234,6 +1233,9 @@ func (t *partitionedTable) GetPartitionColumnIDs() []int64 { } return colIDs } + if t.partitionExpr == nil { + return nil + } partitionCols := expression.ExtractColumns(t.partitionExpr.Expr) colIDs := make([]int64, 0, len(partitionCols)) @@ -1278,15 +1280,15 @@ func (t *partitionedTable) CheckForExchangePartition(ctx sessionctx.Context, pi } // locatePartitionCommon returns the partition idx of the input record. -func (t *partitionedTable) locatePartitionCommon(ctx sessionctx.Context, pi *model.PartitionInfo, partitionExpr *PartitionExpr, num uint64, r []types.Datum) (int, error) { +func (t *partitionedTable) locatePartitionCommon(ctx sessionctx.Context, tp model.PartitionType, partitionExpr *PartitionExpr, num uint64, columnsPartitioned bool, r []types.Datum) (int, error) { var err error var idx int - switch t.meta.Partition.Type { + switch tp { case model.PartitionTypeRange: - if len(pi.Columns) == 0 { - idx, err = t.locateRangePartition(ctx, partitionExpr, r) - } else { + if columnsPartitioned { idx, err = t.locateRangeColumnPartition(ctx, partitionExpr, r) + } else { + idx, err = t.locateRangePartition(ctx, partitionExpr, r) } case model.PartitionTypeHash: // Note that only LIST and RANGE supports REORGANIZE PARTITION @@ -1294,7 +1296,9 @@ func (t *partitionedTable) locatePartitionCommon(ctx sessionctx.Context, pi *mod case model.PartitionTypeKey: idx, err = partitionExpr.LocateKeyPartition(num, r) case model.PartitionTypeList: - idx, err = t.locateListPartition(ctx, partitionExpr, r) + idx, err = partitionExpr.locateListPartition(ctx, r) + case model.PartitionTypeNone: + idx = 0 } if err != nil { return 0, errors.Trace(err) @@ -1304,7 +1308,8 @@ func (t *partitionedTable) locatePartitionCommon(ctx sessionctx.Context, pi *mod func (t *partitionedTable) locatePartition(ctx sessionctx.Context, r []types.Datum) (int64, error) { pi := t.Meta().GetPartitionInfo() - idx, err := t.locatePartitionCommon(ctx, pi, t.partitionExpr, pi.Num, r) + columnsSet := len(t.meta.Partition.Columns) > 0 + idx, err := t.locatePartitionCommon(ctx, pi.Type, t.partitionExpr, pi.Num, columnsSet, r) if err != nil { return 0, errors.Trace(err) } @@ -1313,17 +1318,16 @@ func (t *partitionedTable) locatePartition(ctx sessionctx.Context, r []types.Dat func (t *partitionedTable) locateReorgPartition(ctx sessionctx.Context, r []types.Datum) (int64, error) { pi := t.Meta().GetPartitionInfo() + columnsSet := len(pi.DDLColumns) > 0 // Note that for KEY/HASH partitioning, since we do not support LINEAR, // all partitions will be reorganized, // so we can use the number in Dropping or AddingDefinitions, // depending on current state. - var numParts uint64 + num := len(pi.AddingDefinitions) if pi.DDLState == model.StateDeleteReorganization { - numParts = uint64(len(pi.DroppingDefinitions)) - } else { - numParts = uint64(len(pi.AddingDefinitions)) + num = len(pi.DroppingDefinitions) } - idx, err := t.locatePartitionCommon(ctx, pi, t.reorgPartitionExpr, numParts, r) + idx, err := t.locatePartitionCommon(ctx, pi.DDLType, t.reorgPartitionExpr, uint64(num), columnsSet, r) if err != nil { return 0, errors.Trace(err) } @@ -1375,8 +1379,8 @@ func (t *partitionedTable) locateRangeColumnPartition(ctx sessionctx.Context, pa return idx, nil } -func (t *partitionedTable) locateListPartition(ctx sessionctx.Context, partitionExpr *PartitionExpr, r []types.Datum) (int, error) { - lp := partitionExpr.ForListPruning +func (pe *PartitionExpr) locateListPartition(ctx sessionctx.Context, r []types.Datum) (int, error) { + lp := pe.ForListPruning if len(lp.ColPrunes) == 0 { return lp.locateListPartitionByRow(ctx, r) } @@ -1390,7 +1394,7 @@ func (t *partitionedTable) locateRangePartition(ctx sessionctx.Context, partitio isNull bool err error ) - if col, ok := t.partitionExpr.Expr.(*expression.Column); ok { + if col, ok := partitionExpr.Expr.(*expression.Column); ok { if r[col.Index].IsNull() { isNull = true } @@ -1399,13 +1403,13 @@ func (t *partitionedTable) locateRangePartition(ctx sessionctx.Context, partitio evalBuffer := t.evalBufferPool.Get().(*chunk.MutRow) defer t.evalBufferPool.Put(evalBuffer) evalBuffer.SetDatums(r...) - val, isNull, err = t.partitionExpr.Expr.EvalInt(ctx, evalBuffer.ToRow()) + val, isNull, err = partitionExpr.Expr.EvalInt(ctx, evalBuffer.ToRow()) if err != nil { return 0, err } ret = val } - unsigned := mysql.HasUnsignedFlag(t.partitionExpr.Expr.GetType().GetFlag()) + unsigned := mysql.HasUnsignedFlag(partitionExpr.Expr.GetType().GetFlag()) ranges := partitionExpr.ForRangePruning length := len(ranges.LessThan) pos := sort.Search(length, func(i int) bool { @@ -1420,6 +1424,7 @@ func (t *partitionedTable) locateRangePartition(ctx sessionctx.Context, partitio if pos < 0 || pos >= length { // The data does not belong to any of the partition returns `table has no partition for value %s`. var valueMsg string + // TODO: Test with ALTER TABLE t PARTITION BY with a different expression / type if t.meta.Partition.Expr != "" { e, err := expression.ParseSimpleExprWithTableInfo(ctx, t.meta.Partition.Expr, t.meta) if err == nil { @@ -1502,10 +1507,18 @@ func GetReorganizedPartitionedTable(t table.Table) (table.PartitionedTable, erro return nil, dbterror.ErrUnsupportedReorganizePartition.GenWithStackByArgs() } tblInfo := t.Meta().Clone() - tblInfo.Partition.Definitions = tblInfo.Partition.AddingDefinitions - tblInfo.Partition.AddingDefinitions = nil - tblInfo.Partition.DroppingDefinitions = nil - tblInfo.Partition.Num = uint64(len(tblInfo.Partition.Definitions)) + pi := tblInfo.Partition + pi.Definitions = pi.AddingDefinitions + pi.Num = uint64(len(pi.Definitions)) + pi.AddingDefinitions = nil + pi.DroppingDefinitions = nil + + // Reorganized status, use the new values + pi.Type = pi.DDLType + pi.Expr = pi.DDLExpr + pi.Columns = pi.DDLColumns + tblInfo.ID = pi.NewTableID + constraints, err := table.LoadCheckConstraint(tblInfo) if err != nil { return nil, err @@ -1514,7 +1527,6 @@ func GetReorganizedPartitionedTable(t table.Table) (table.PartitionedTable, erro initTableCommon(&tc, tblInfo, tblInfo.ID, t.Cols(), t.Allocators(nil), constraints) // and rebuild the partitioning structure - return newPartitionedTable(&tc, tblInfo) } diff --git a/table/tables/partition_test.go b/table/tables/partition_test.go index 199a331fda358..257a1049d0a89 100644 --- a/table/tables/partition_test.go +++ b/table/tables/partition_test.go @@ -23,7 +23,6 @@ import ( "testing" gotime "time" - "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" mysql "github.com/pingcap/tidb/errno" @@ -400,13 +399,9 @@ func TestLocatePartitionSingleColumn(t *testing.T) { } func TestLocatePartition(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/planner/core/forceDynamicPrune") store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.MustExec("set @@session.tidb_enable_list_partition = ON;") - tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic';") tk.MustExec("drop table if exists t;") tk.MustExec(`CREATE TABLE t ( @@ -416,7 +411,9 @@ func TestLocatePartition(t *testing.T) { PARTITION BY LIST COLUMNS(type) (PARTITION push_event VALUES IN ("PushEvent"), PARTITION watch_event VALUES IN ("WatchEvent") - );`) + )`) + tk.MustExec(`insert into t values (1,"PushEvent"),(2,"WatchEvent"),(3, "WatchEvent")`) + tk.MustExec(`analyze table t`) tk1 := testkit.NewTestKit(t, store) tk2 := testkit.NewTestKit(t, store) @@ -426,7 +423,10 @@ func TestLocatePartition(t *testing.T) { wg := util.WaitGroupWrapper{} exec := func(tk0 *testkit.TestKit) { tk0.MustExec("use test") - tk0.MustQuery("desc select id, type from t where type = 'WatchEvent';").Check(testkit.Rows("TableReader_7 10.00 root partition:watch_event data:Selection_6]\n[└─Selection_6 10.00 cop[tikv] eq(test.t.type, \"WatchEvent\")]\n[ └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo")) + tk0.MustQuery("explain format = 'brief' select id, type from t where type = 'WatchEvent';").Check(testkit.Rows(""+ + `TableReader 2.00 root partition:watch_event data:Selection`, + `└─Selection 2.00 cop[tikv] eq(test.t.type, "WatchEvent")`, + ` └─TableFullScan 3.00 cop[tikv] table:t keep order:false`)) } run := func(num int) { @@ -915,7 +915,7 @@ func executePartTableCase(t *testing.T, tk *testkit.TestKit, testCases []partTab for i, testCase := range testCases { // create table ... partition by key ... ddlSQL := createSQL + testCase.partitionbySQL - fmt.Println(i, ":", ddlSQL) + logutil.BgLogger().Info("Partition DDL test", zap.Int("i", i), zap.String("ddlSQL", ddlSQL)) executeSQLWrapper(t, tk, ddlSQL) // insert data for _, insertsql := range insertSQLs { @@ -923,7 +923,7 @@ func executePartTableCase(t *testing.T, tk *testkit.TestKit, testCases []partTab } // execute testcases for j, selInfo := range testCase.selectInfo { - fmt.Println(j, ":", selInfo.selectSQL) + logutil.BgLogger().Info("Select", zap.Int("j", j), zap.String("selectSQL", selInfo.selectSQL)) tk.MustQuery(selInfo.selectSQL).Check(testkit.Rows(strconv.Itoa(selInfo.rowCount))) if selInfo.executeExplain { result := tk.MustQuery("EXPLAIN " + selInfo.selectSQL) @@ -1355,11 +1355,11 @@ func TestKeyPartitionTableBasic(t *testing.T) { } for i, testCase := range testCases { - fmt.Println(i, ":", testCase.createSQL) + logutil.BgLogger().Info("Partition DDL test", zap.Int("i", i), zap.String("createSQL", testCase.createSQL)) executeSQLWrapper(t, tk, testCase.createSQL) executeSQLWrapper(t, tk, testCase.insertSQL) for j, selInfo := range testCase.selectInfo { - fmt.Println(j, ":", selInfo.selectSQL) + logutil.BgLogger().Info("Select", zap.Int("j", j), zap.String("selectSQL", selInfo.selectSQL)) tk.MustQuery(selInfo.selectSQL).Check(testkit.Rows(strconv.Itoa(selInfo.rowCount))) if selInfo.executeExplain { result := tk.MustQuery("EXPLAIN " + selInfo.selectSQL) @@ -2490,10 +2490,8 @@ func TestKeyPartitionTableDDL(t *testing.T) { "PARTITION BY KEY(col3) PARTITIONS 4") tk.MustExec("INSERT INTO tkey16 values(1,1,1,1),(1,1,2,2),(3,3,3,3),(3,3,4,3),(4,4,4,4),(5,5,5,5),(6,6,6,6),(7,7,7,7),(8,8,8,8),(9,9,9,9),(10,10,10,5),(11,11,11,6),(12,12,12,12),(13,13,13,13),(14,14,14,14)") - err := tk.ExecToErr("ALTER TABLE tkey15 PARTITION BY KEY(col3) PARTITIONS 4") - require.Regexp(t, "alter table partition is unsupported", err) tk.MustExec("ALTER TABLE tkey14 ADD PARTITION PARTITIONS 1") - err = tk.ExecToErr("ALTER TABLE tkey14 DROP PARTITION p4") + err := tk.ExecToErr("ALTER TABLE tkey14 DROP PARTITION p4") require.Regexp(t, "DROP PARTITION can only be used on RANGE/LIST partitions", err) tk.MustExec("ALTER TABLE tkey14 TRUNCATE PARTITION p3") tk.MustQuery("SELECT COUNT(*) FROM tkey14 partition(p3)").Check(testkit.Rows("0")) @@ -2516,8 +2514,8 @@ func TestKeyPartitionTableDDL(t *testing.T) { require.Regexp(t, "Unsupported reorganize partition", err) err = tk.ExecToErr("ALTER TABLE tkey16 REORGANIZE PARTITION p0 INTO (PARTITION p4)") require.Regexp(t, "Unsupported reorganize partition", err) - err = tk.ExecToErr("ALTER TABLE tkey16 REMOVE PARTITIONING") - require.Regexp(t, "Unsupported remove partitioning", err) + tk.MustExec("ALTER TABLE tkey15 PARTITION BY KEY(col3) PARTITIONS 4") + tk.MustExec("ALTER TABLE tkey16 REMOVE PARTITIONING") tk.MustExec("CREATE TABLE tkey17 (" + "id INT NOT NULL PRIMARY KEY," + @@ -2574,12 +2572,358 @@ func (c *testCallback) OnJobRunBefore(job *model.Job) { } } +// TODO: do extensive test for LIST [COLUMNS] +// TODO: Either skip this, move it to a separate directory for big tests +// or see if there are ways to speed this up :) +// Leaving the test here, for reference and completeness testing +func TestPartitionByIntListExtensivePart(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "PartitionByIntListExtensive" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use " + schemaName) + + tBase := `(lp tinyint unsigned, a int unsigned, b varchar(255) collate utf8mb4_general_ci, c int, d datetime, e timestamp, f double, g text, key (b), key (c,b), key (d,c), key(e), primary key (a, lp))` + t2Str := `create table t2 ` + tBase + tStr := `create table t ` + tBase + + rows := 1000 + pkInserts := 20 + pkUpdates := 20 + pkDeletes := 10 // Enough to delete half of what is inserted? + tStart := []string{ + // Non partitioned + tStr, + // RANGE COLUMNS + tStr + ` partition by list (lp) (partition p0 values in (0,6),partition p1 values in (1), partition p2 values in (2), partition p3 values in (3), partition p4 values in (4,5))`, + // KEY + tStr + ` partition by key(a) partitions 5`, + // HASH + tStr + ` partition by hash(a) partitions 5`, + // HASH with function + tStr + ` partition by hash(a DIV 3) partitions 5`, + } + quarterUintRange := 1 << 30 + quarterUintRangeStr := fmt.Sprintf("%d", quarterUintRange) + halfUintRangeStr := fmt.Sprintf("%d", 2*quarterUintRange) + threeQuarterUintRangeStr := fmt.Sprintf("%d", 3*quarterUintRange) + tAlter := []string{ + // LIST + `alter table t partition by list (lp) (partition p0 values in (2), partition p1 values in (1,3,5), partition p2 values in (0,4,6))`, + `alter table t partition by list (lp) (partition p3 values in (3), partition p4 values in (4), partition p2 values in (2), partition p6 values in (6), partition p5 values in (5), partition p1 values in (1), partition p0 values in (0))`, + // LIST COLUMNS + `alter table t partition by list columns (lp) (partition p0 values in (2), partition p1 values in (1,3,5), partition p2 values in (0,4,6))`, + `alter table t partition by list columns (lp) (partition p3 values in (3), partition p4 values in (4), partition p2 values in (2), partition p6 values in (6), partition p5 values in (5), partition p1 values in (1), partition p0 values in (0))`, + // RANGE COLUMNS + `alter table t partition by range (a) (partition pFirst values less than (` + halfUintRangeStr + `), partition pLast values less than (MAXVALUE))`, + // RANGE COLUMNS + `alter table t partition by range (a) (partition pFirst values less than (` + quarterUintRangeStr + `),` + + `partition pLowMid values less than (` + halfUintRangeStr + `),` + + `partition pHighMid values less than (` + threeQuarterUintRangeStr + `),` + + `partition pLast values less than (maxvalue))`, + // KEY + `alter table t partition by key(a) partitions 7`, + `alter table t partition by key(a) partitions 3`, + // Hash + `alter table t partition by hash(a) partitions 7`, + `alter table t partition by hash(a) partitions 3`, + // Hash + `alter table t partition by hash(a DIV 13) partitions 7`, + `alter table t partition by hash(a DIV 13) partitions 3`, + } + + seed := gotime.Now().UnixNano() + logutil.BgLogger().Info("Seeding rand", zap.Int64("seed", seed)) + reorgRand := rand.New(rand.NewSource(seed)) + for _, createSQL := range tStart { + for _, alterSQL := range tAlter { + tk.MustExec(createSQL) + tk.MustExec(t2Str) + getNewPK := getNewIntPK() + getValues := getInt7ValuesFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, alterSQL, rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) + tk.MustExec(`drop table t`) + tk.MustExec(`drop table t2`) + } + } + for _, createSQL := range tStart[1:] { + tk.MustExec(createSQL) + tk.MustExec(t2Str) + getNewPK := getNewIntPK() + getValues := getInt7ValuesFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, "alter table t remove partitioning", rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) + tk.MustExec(`drop table t`) + tk.MustExec(`drop table t2`) + } +} + +func getInt7ValuesFunc() func(string, bool, *rand.Rand) string { + cnt := 0 + return func(pk string, asAssignment bool, reorgRand *rand.Rand) string { + s := `(%d, %s, '%s', %d, '%s', '%s', %f, '%s')` + if asAssignment { + s = `lp = %d, a = %s, b = '%s', c = %d, d = '%s', e = '%s', f = %f, g = '%s'` + } + cnt++ + lp, err := strconv.Atoi(pk) + if err != nil { + lp = 0 + } + return fmt.Sprintf(s, + lp%7, + pk, + randStr(reorgRand.Intn(19), reorgRand), + cnt, //reorgRand.Int31(), + gotime.Unix(413487608+int64(reorgRand.Intn(1705689644)), 0).Format("2006-01-02T15:04:05"), + gotime.Unix(413487608+int64(reorgRand.Intn(1705689644)), 0).Format("2006-01-02T15:04:05"), + reorgRand.Float64(), + randStr(512+reorgRand.Intn(1024), reorgRand)) + } +} + +// TODO: Either skip this, move it to a separate directory for big tests +// or see if there are ways to speed this up :) +// Leaving the test here, for reference and completeness testing +func TestPartitionByIntExtensivePart(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "PartitionByIntExtensive" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use " + schemaName) + + tBase := `(a int unsigned, b varchar(255) collate utf8mb4_general_ci, c int, d datetime, e timestamp, f double, g text, primary key (a), key (b), key (c,b), key (d,c), key(e))` + t2Str := `create table t2 ` + tBase + tStr := `create table t ` + tBase + + rows := 1000 + pkInserts := 20 + pkUpdates := 20 + pkDeletes := 10 // Enough to delete half of what is inserted? + thirdUintRange := 1 << 32 / 2 + thirdUintRangeStr := fmt.Sprintf("%d", thirdUintRange) + twoThirdUintRangeStr := fmt.Sprintf("%d", 2*thirdUintRange) + tStart := []string{ + // Non partitioned + tStr, + // RANGE COLUMNS + tStr + ` partition by range (a) (partition pFirst values less than (` + thirdUintRangeStr + `),` + + `partition pMid values less than (` + twoThirdUintRangeStr + `), partition pLast values less than (maxvalue))`, + // KEY + tStr + ` partition by key(a) partitions 5`, + // HASH + tStr + ` partition by hash(a) partitions 5`, + // HASH with function + tStr + ` partition by hash(a DIV 3) partitions 5`, + } + quarterUintRange := 1 << 30 + quarterUintRangeStr := fmt.Sprintf("%d", quarterUintRange) + halfUintRangeStr := fmt.Sprintf("%d", 2*quarterUintRange) + threeQuarterUintRangeStr := fmt.Sprintf("%d", 3*quarterUintRange) + tAlter := []string{ + // RANGE COLUMNS + `alter table t partition by range (a) (partition pFirst values less than (` + halfUintRangeStr + `), partition pLast values less than (MAXVALUE))`, + // RANGE COLUMNS + `alter table t partition by range (a) (partition pFirst values less than (` + quarterUintRangeStr + `),` + + `partition pLowMid values less than (` + halfUintRangeStr + `),` + + `partition pHighMid values less than (` + threeQuarterUintRangeStr + `),` + + `partition pLast values less than (maxvalue))`, + // KEY + `alter table t partition by key(a) partitions 7`, + `alter table t partition by key(a) partitions 3`, + // Hash + `alter table t partition by hash(a) partitions 7`, + `alter table t partition by hash(a) partitions 3`, + // Hash + `alter table t partition by hash(a DIV 13) partitions 7`, + `alter table t partition by hash(a DIV 13) partitions 3`, + } + + seed := gotime.Now().UnixNano() + logutil.BgLogger().Info("Seeding rand", zap.Int64("seed", seed)) + reorgRand := rand.New(rand.NewSource(seed)) + for _, createSQL := range tStart { + for _, alterSQL := range tAlter { + tk.MustExec(createSQL) + tk.MustExec(t2Str) + getNewPK := getNewIntPK() + getValues := getIntValuesFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, alterSQL, rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) + tk.MustExec(`drop table t`) + tk.MustExec(`drop table t2`) + } + } + for _, createSQL := range tStart[1:] { + tk.MustExec(createSQL) + tk.MustExec(t2Str) + getNewPK := getNewIntPK() + getValues := getIntValuesFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, "alter table t remove partitioning", rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) + tk.MustExec(`drop table t`) + tk.MustExec(`drop table t2`) + } +} + +func getNewIntPK() func(map[string]struct{}, string, *rand.Rand) string { + return func(m map[string]struct{}, suf string, reorgRand *rand.Rand) string { + uintPK := reorgRand.Uint32() + newPK := strconv.FormatUint(uint64(uintPK), 10) + for _, ok := m[newPK]; ok; { + uintPK = reorgRand.Uint32() + newPK = strconv.FormatUint(uint64(uintPK), 10) + _, ok = m[newPK] + } + m[newPK] = struct{}{} + return newPK + } +} + +func getIntValuesFunc() func(string, bool, *rand.Rand) string { + cnt := 0 + return func(pk string, asAssignment bool, reorgRand *rand.Rand) string { + s := `(%s, '%s', %d, '%s', '%s', %f, '%s')` + if asAssignment { + s = `a = %s, b = '%s', c = %d, d = '%s', e = '%s', f = %f, g = '%s'` + } + cnt++ + return fmt.Sprintf(s, + pk, + randStr(reorgRand.Intn(19), reorgRand), + cnt, //reorgRand.Int31(), + gotime.Unix(413487608+int64(reorgRand.Intn(1705689644)), 0).Format("2006-01-02T15:04:05"), + gotime.Unix(413487608+int64(reorgRand.Intn(1705689644)), 0).Format("2006-01-02T15:04:05"), + reorgRand.Float64(), + randStr(512+reorgRand.Intn(1024), reorgRand)) + } +} + +func TestRangePartitionByRange(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "RangePartitionByRange" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk.MustExec(`create table t (a int) partition by range(a) (partition p0 values less than (0), partition p1M values less than (1000000))`) + tk.MustExec(`insert into t values (-1),(0),(1)`) + tk.MustExec(`alter table t partition by range(a) (partition p0 values less than (0), partition p1M values less than (1000000))`) + tk.MustExec(`alter table t remove partitioning`) + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows("-1", "0", "1")) +} + +// TODO: Either skip this, move it to a separate directory for big tests +// or see if there are ways to speed this up :) +// Leaving the test here, for reference and completeness testing +func TestPartitionByExtensivePart(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "PartitionByExtensive" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use " + schemaName) + + tBase := `(a varchar(255) collate utf8mb4_unicode_ci, b varchar(255) collate utf8mb4_general_ci, c int, d datetime, e timestamp, f double, g text, primary key (a), key (b), key (c,b), key (d,c), key(e))` + t2Str := `create table t2 ` + tBase + tStr := `create table t ` + tBase + + rows := 1000 + pkInserts := 20 + pkUpdates := 20 + pkDeletes := 10 // Enough to delete half of what is inserted? + tStart := []string{ + // Non partitioned + tStr, + // RANGE COLUMNS + tStr + ` partition by range columns (a) (partition pNull values less than (""), partition pM values less than ("M"), partition pLast values less than (maxvalue))`, + // KEY + tStr + ` partition by key(a) partitions 5`, + } + showCreateStr := "t CREATE TABLE `t` (\n" + + " `a` varchar(255) COLLATE utf8mb4_unicode_ci NOT NULL,\n" + + " `b` varchar(255) COLLATE utf8mb4_general_ci DEFAULT NULL,\n" + + " `c` int(11) DEFAULT NULL,\n" + + " `d` datetime DEFAULT NULL,\n" + + " `e` timestamp NULL DEFAULT NULL,\n" + + " `f` double DEFAULT NULL,\n" + + " `g` text DEFAULT NULL,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`),\n" + + " KEY `c` (`c`,`b`),\n" + + " KEY `d` (`d`,`c`),\n" + + " KEY `e` (`e`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + tAlter := []struct{ alter, result string }{ + { + // RANGE COLUMNS + alter: `alter table t partition by range columns (a) (partition pH values less than ("H"), partition pLast values less than (MAXVALUE))`, + result: showCreateStr + + "PARTITION BY RANGE COLUMNS(`a`)\n" + + "(PARTITION `pH` VALUES LESS THAN ('H'),\n" + + " PARTITION `pLast` VALUES LESS THAN (MAXVALUE))", + }, + { + // RANGE COLUMNS + alter: `alter table t partition by range columns (a) (partition pNull values less than (""), partition pG values less than ("G"), partition pR values less than ("R"), partition pLast values less than (maxvalue))`, + result: showCreateStr + + "PARTITION BY RANGE COLUMNS(`a`)\n" + + "(PARTITION `pNull` VALUES LESS THAN (''),\n" + + " PARTITION `pG` VALUES LESS THAN ('G'),\n" + + " PARTITION `pR` VALUES LESS THAN ('R'),\n" + + " PARTITION `pLast` VALUES LESS THAN (MAXVALUE))", + }, + // KEY + { + alter: `alter table t partition by key(a) partitions 7`, + result: showCreateStr + + "PARTITION BY KEY (`a`) PARTITIONS 7", + }, + { + alter: `alter table t partition by key(a) partitions 3`, + result: showCreateStr + + "PARTITION BY KEY (`a`) PARTITIONS 3", + }, + } + + seed := gotime.Now().UnixNano() + logutil.BgLogger().Info("Seeding rand", zap.Int64("seed", seed)) + reorgRand := rand.New(rand.NewSource(seed)) + for _, createSQL := range tStart { + for _, alterSQL := range tAlter { + tk.MustExec(createSQL) + tk.MustExec(t2Str) + getNewPK := getNewStringPK() + getValues := getValuesFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, alterSQL.alter, rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) + res := tk.MustQuery(`show create table t`) + res.AddComment("create SQL: " + createSQL + "\nalterSQL: " + alterSQL.alter) + res.Check(testkit.Rows(alterSQL.result)) + tk.MustExec(`drop table t`) + tk.MustExec(`drop table t2`) + } + } + + for _, createSQL := range tStart[1:] { + tk.MustExec(createSQL) + tk.MustExec(t2Str) + getNewPK := getNewStringPK() + getValues := getValuesFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, "alter table t remove partitioning", rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) + tk.MustExec(`drop table t`) + tk.MustExec(`drop table t2`) + } +} + func TestReorgPartExtensivePart(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) schemaName := "ReorgPartExtensive" tk.MustExec("create database " + schemaName) tk.MustExec("use " + schemaName) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use " + schemaName) // TODO: Handle different column types? // TODO: Handle index for different column types / combinations as well? @@ -2632,22 +2976,21 @@ func TestReorgPartExtensivePart(t *testing.T) { " KEY `e` (`e`)\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) - dom := domain.GetDomain(tk.Session()) - originHook := dom.DDL().GetHook() - defer dom.DDL().SetHook(originHook) - hook := newTestCallBack(t, dom) - dom.DDL().SetHook(hook) - rows := 10000 pkInserts := 200 pkUpdates := 200 pkDeletes := 100 // Enough to delete half of what is inserted? - pkMap := make(map[string]struct{}, rows) - pkArray := make([]string, 0, len(pkMap)) + alterStr := `alter table t reorganize partition pNull, pM, pLast into (partition pI values less than ("I"), partition pQ values less than ("q"), partition pLast values less than (MAXVALUE))` seed := rand.Int63() logutil.BgLogger().Info("Seeding rand", zap.Int64("seed", seed)) reorgRand := rand.New(rand.NewSource(seed)) - getNewPK := func(m map[string]struct{}, suf string) string { + getNewPK := getNewStringPK() + getValues := getValuesFunc() + checkDMLInAllStates(t, tk, tk2, schemaName, alterStr, rows, pkInserts, pkUpdates, pkDeletes, reorgRand, getNewPK, getValues) +} + +func getNewStringPK() func(map[string]struct{}, string, *rand.Rand) string { + return func(m map[string]struct{}, suf string, reorgRand *rand.Rand) string { newPK := randStr(2+reorgRand.Intn(5), reorgRand) + suf lowerPK := strings.ToLower(newPK) for _, ok := m[lowerPK]; ok; { @@ -2658,8 +3001,11 @@ func TestReorgPartExtensivePart(t *testing.T) { m[lowerPK] = struct{}{} return newPK } +} + +func getValuesFunc() func(string, bool, *rand.Rand) string { cnt := 0 - getValues := func(pk string, asAssignment bool) string { + return func(pk string, asAssignment bool, reorgRand *rand.Rand) string { s := `('%s', '%s', %d, '%s', '%s', %f, '%s')` if asAssignment { s = `a = '%s', b = '%s', c = %d, d = '%s', e = '%s', f = %f, g = '%s'` @@ -2674,11 +3020,26 @@ func TestReorgPartExtensivePart(t *testing.T) { reorgRand.Float64(), randStr(512+reorgRand.Intn(1024), reorgRand)) } +} + +func checkDMLInAllStates(t *testing.T, tk, tk2 *testkit.TestKit, schemaName, alterStr string, + rows, pkInserts, pkUpdates, pkDeletes int, + reorgRand *rand.Rand, + getNewPK func(map[string]struct{}, string, *rand.Rand) string, + getValues func(string, bool, *rand.Rand) string) { + dom := domain.GetDomain(tk.Session()) + originHook := dom.DDL().GetHook() + defer dom.DDL().SetHook(originHook) + hook := newTestCallBack(t, dom) + dom.DDL().SetHook(hook) + + pkMap := make(map[string]struct{}, rows) + pkArray := make([]string, 0, len(pkMap)) // Generate a start set: for i := 0; i < rows; i++ { - pk := getNewPK(pkMap, "-o") + pk := getNewPK(pkMap, "-o", reorgRand) pkArray = append(pkArray, pk) - values := getValues(pk, false) + values := getValues(pk, false, reorgRand) tk.MustExec(`insert into t values ` + values) tk.MustExec(`insert into t2 values ` + values) } @@ -2705,8 +3066,6 @@ func TestReorgPartExtensivePart(t *testing.T) { // update // delete // Note: update the PK so it moves between different before and after partitions - tk2 := testkit.NewTestKit(t, store) - tk2.MustExec("use " + schemaName) tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) currentState := model.StateNone @@ -2739,11 +3098,11 @@ func TestReorgPartExtensivePart(t *testing.T) { insPK := make([]string, 0, pkInserts) values := make([]string, 0, pkInserts) for i := 0; i < pkInserts; i += 2 { - pk := getNewPK(pkMap, "-i0") + pk := getNewPK(pkMap, "-i0", reorgRand) logutil.BgLogger().Debug("insert1", zap.String("pk", pk)) pkArray = append(pkArray, pk) insPK = append(insPK, pk) - values = append(values, getValues(pk, false)) + values = append(values, getValues(pk, false, reorgRand)) } if len(pkMap) != len(pkArray) { panic("Different length!!!") @@ -2768,11 +3127,11 @@ func TestReorgPartExtensivePart(t *testing.T) { values = values[:0] for i := 1; i < pkInserts; i += 2 { - pk := getNewPK(pkMap, "-i1") + pk := getNewPK(pkMap, "-i1", reorgRand) logutil.BgLogger().Debug("insert2", zap.String("pk", pk)) pkArray = append(pkArray, pk) insPK = append(insPK, pk) - values = append(values, getValues(pk, false)) + values = append(values, getValues(pk, false, reorgRand)) } hookErr = tk2.ExecToErr(`insert into t values ` + strings.Join(values, ",")) if hookErr != nil { @@ -2815,11 +3174,11 @@ func TestReorgPartExtensivePart(t *testing.T) { oldPK := insPK[insIdx] lowerPK := strings.ToLower(oldPK) delete(pkMap, lowerPK) - newPK := getNewPK(pkMap, "-u0") + newPK := getNewPK(pkMap, "-u0", reorgRand) insPK[insIdx] = newPK idx := len(pkArray) - len(insPK) + insIdx pkArray[idx] = newPK - value := getValues(newPK, true) + value := getValues(newPK, true, reorgRand) logutil.BgLogger().Debug("update1", zap.String("old", oldPK), zap.String("value", value)) hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) @@ -2834,7 +3193,7 @@ func TestReorgPartExtensivePart(t *testing.T) { // Also do some non-pk column updates! insIdx = reorgRand.Intn(len(insPK)) oldPK = insPK[insIdx] - value = getValues(oldPK, true) + value = getValues(oldPK, true, reorgRand) hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) if hookErr != nil { @@ -2863,11 +3222,11 @@ func TestReorgPartExtensivePart(t *testing.T) { oldPK := insPK[insIdx] lowerPK := strings.ToLower(oldPK) delete(pkMap, lowerPK) - newPK := getNewPK(pkMap, "-u1") + newPK := getNewPK(pkMap, "-u1", reorgRand) insPK[insIdx] = newPK idx := len(pkArray) - len(insPK) + insIdx pkArray[idx] = newPK - value := getValues(newPK, true) + value := getValues(newPK, true, reorgRand) logutil.BgLogger().Debug("update2", zap.String("old", oldPK), zap.String("value", value)) hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) @@ -2883,7 +3242,7 @@ func TestReorgPartExtensivePart(t *testing.T) { // Note: if PK changes it does RemoveRecord + AddRecord insIdx = reorgRand.Intn(len(insPK)) oldPK = insPK[insIdx] - value = getValues(oldPK, true) + value = getValues(oldPK, true, reorgRand) hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) if hookErr != nil { @@ -2913,9 +3272,9 @@ func TestReorgPartExtensivePart(t *testing.T) { oldPK := pkArray[idx] lowerPK := strings.ToLower(oldPK) delete(pkMap, lowerPK) - newPK := getNewPK(pkMap, "-u2") + newPK := getNewPK(pkMap, "-u2", reorgRand) pkArray[idx] = newPK - value := getValues(newPK, true) + value := getValues(newPK, true, reorgRand) logutil.BgLogger().Debug("update3", zap.String("old", oldPK), zap.String("value", value)) hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) @@ -2930,7 +3289,7 @@ func TestReorgPartExtensivePart(t *testing.T) { // Also do some non-pk column updates! idx = reorgRand.Intn(len(pkArray) - len(insPK)) oldPK = pkArray[idx] - value = getValues(oldPK, true) + value = getValues(oldPK, true, reorgRand) hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) if hookErr != nil { @@ -2959,9 +3318,9 @@ func TestReorgPartExtensivePart(t *testing.T) { oldPK := pkArray[idx] lowerPK := strings.ToLower(oldPK) delete(pkMap, lowerPK) - newPK := getNewPK(pkMap, "-u3") + newPK := getNewPK(pkMap, "-u3", reorgRand) pkArray[idx] = newPK - value := getValues(newPK, true) + value := getValues(newPK, true, reorgRand) logutil.BgLogger().Debug("update4", zap.String("old", oldPK), zap.String("value", value)) hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) @@ -2976,7 +3335,7 @@ func TestReorgPartExtensivePart(t *testing.T) { // Also do some non-pk column updates! idx = reorgRand.Intn(len(pkArray) - len(insPK)) oldPK = pkArray[idx] - value = getValues(oldPK, true) + value = getValues(oldPK, true, reorgRand) hookErr = tk2.ExecToErr(`update t set ` + value + ` where a = "` + oldPK + `"`) if hookErr != nil { @@ -3156,7 +3515,7 @@ func TestReorgPartExtensivePart(t *testing.T) { zap.Int("rows", len(pkMap)), zap.Stringer("SchemaState", job.SchemaState)) } } - tk.MustExec(`alter table t reorganize partition pNull, pM, pLast into (partition pI values less than ("I"), partition pQ values less than ("q"), partition pLast values less than (MAXVALUE))`) + tk.MustExec(alterStr) require.NoError(t, hookErr) tk.MustExec(`admin check table t`) tk.MustExec(`admin check table t2`) From 69996953a6cfe7622de1af0d9608cf9479a5011a Mon Sep 17 00:00:00 2001 From: fzzf678 <108643977+fzzf678@users.noreply.github.com> Date: Fri, 18 Aug 2023 17:54:39 +0800 Subject: [PATCH 17/32] *: modify header of some source file (#46219) --- ddl/constraint.go | 2 +- ddl/constraint_test.go | 2 +- table/constraint.go | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/ddl/constraint.go b/ddl/constraint.go index e4b8856f03d1f..5ddbbe3a87700 100644 --- a/ddl/constraint.go +++ b/ddl/constraint.go @@ -1,4 +1,4 @@ -// Copyright 2023-2023 PingCAP Xingchen (Beijing) Technology Co., Ltd. +// Copyright 2023-2023 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/ddl/constraint_test.go b/ddl/constraint_test.go index d278a71e7d947..49181c6b45ea4 100644 --- a/ddl/constraint_test.go +++ b/ddl/constraint_test.go @@ -1,4 +1,4 @@ -// Copyright 2023-2023 PingCAP Xingchen (Beijing) Technology Co., Ltd. +// Copyright 2023-2023 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/table/constraint.go b/table/constraint.go index 33ad0d476d2df..16d79fb7f904e 100644 --- a/table/constraint.go +++ b/table/constraint.go @@ -1,4 +1,4 @@ -// Copyright 2023-2023 PingCAP Xingchen (Beijing) Technology Co., Ltd. +// Copyright 2023-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. From 0b2d62bcf6f32ba7520cc284375490203c4c41de Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Fri, 18 Aug 2023 21:56:01 +0800 Subject: [PATCH 18/32] executor: enable LFU stats cache for test (#46226) ref pingcap/tidb#46158 --- executor/main_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/executor/main_test.go b/executor/main_test.go index 3749e2e273152..4e1e3fedf1f83 100644 --- a/executor/main_test.go +++ b/executor/main_test.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/testkit/testmain" "github.com/pingcap/tidb/testkit/testsetup" @@ -51,6 +52,7 @@ func TestMain(m *testing.M) { conf.Experimental.AllowsExpressionIndex = true }) tikv.EnableFailpoints() + variable.StatsCacheMemQuota.Store(5000) opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"), From ab4d6ad71cccbfcc12cc14a5542f7e955de4baab Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Mon, 21 Aug 2023 10:11:03 +0800 Subject: [PATCH 19/32] *: simple code (#46247) --- statistics/cmsketch.go | 10 ++-------- statistics/cmsketch_test.go | 4 ++-- statistics/handle/handle.go | 5 ++++- 3 files changed, 8 insertions(+), 11 deletions(-) diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index 3c6d9ee566717..f94d00ca08fa5 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -919,14 +919,13 @@ func checkEmptyTopNs(topNs []*TopN) bool { } // SortTopnMeta sort topnMeta -func SortTopnMeta(topnMetas []TopNMeta) []TopNMeta { +func SortTopnMeta(topnMetas []TopNMeta) { slices.SortFunc(topnMetas, func(i, j TopNMeta) int { if i.Count != j.Count { return cmp.Compare(j.Count, i.Count) } return bytes.Compare(i.Encoded, j.Encoded) }) - return topnMetas } // GetMergedTopNFromSortedSlice returns merged topn @@ -935,12 +934,7 @@ func GetMergedTopNFromSortedSlice(sorted []TopNMeta, n uint32) (*TopN, []TopNMet } func getMergedTopNFromSortedSlice(sorted []TopNMeta, n uint32) (*TopN, []TopNMeta) { - slices.SortFunc(sorted, func(i, j TopNMeta) int { - if i.Count != j.Count { - return cmp.Compare(j.Count, i.Count) - } - return bytes.Compare(i.Encoded, j.Encoded) - }) + SortTopnMeta(sorted) n = mathutil.Min(uint32(len(sorted)), n) var finalTopN TopN diff --git a/statistics/cmsketch_test.go b/statistics/cmsketch_test.go index e715235c193af..8d165b7b54d68 100644 --- a/statistics/cmsketch_test.go +++ b/statistics/cmsketch_test.go @@ -347,8 +347,8 @@ func TestSortTopnMeta(t *testing.T) { Encoded: []byte("b"), Count: 2, }} - sortedData := SortTopnMeta(data) - require.Equal(t, uint64(2), sortedData[0].Count) + SortTopnMeta(data) + require.Equal(t, uint64(2), data[0].Count) } func TestMergePartTopN2GlobalTopNWithHists(t *testing.T) { diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index a862b03d949b4..17402392fb548 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -970,7 +970,10 @@ func MergeGlobalStatsTopNByConcurrency(mergeConcurrency, mergeBatchSize int, wra } globalTopN, popedTopn := statistics.GetMergedTopNFromSortedSlice(sorted, n) - return globalTopN, statistics.SortTopnMeta(append(leftTopn, popedTopn...)), wrapper.AllHg, nil + + result := append(leftTopn, popedTopn...) + statistics.SortTopnMeta(result) + return globalTopN, result, wrapper.AllHg, nil } func (h *Handle) getTableByPhysicalID(is infoschema.InfoSchema, physicalID int64) (table.Table, bool) { From 8c1d2c851e1ce31a99d101c8d1c123438c17b0d1 Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 21 Aug 2023 12:59:02 +0800 Subject: [PATCH 20/32] session: remove pause all jobs when upgrading (#46234) close pingcap/tidb#46227, close pingcap/tidb#46228 --- session/bootstrap.go | 25 ------------- .../bootstraptest/bootstrap_upgrade_test.go | 37 ++++++++++++++----- 2 files changed, 27 insertions(+), 35 deletions(-) diff --git a/session/bootstrap.go b/session/bootstrap.go index d31820a4d6a12..62d9cd263ce8b 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -1282,31 +1282,6 @@ func SyncUpgradeState(s Session) error { time.Sleep(interval) } - retryTimes = 60 - interval = 500 * time.Millisecond - for i := 0; i < retryTimes; i++ { - jobErrs, err := ddl.PauseAllJobsBySystem(s) - if err == nil && len(jobErrs) == 0 { - break - } - jobErrStrs := make([]string, 0, len(jobErrs)) - for _, jobErr := range jobErrs { - if dbterror.ErrPausedDDLJob.Equal(jobErr) { - continue - } - jobErrStrs = append(jobErrStrs, jobErr.Error()) - } - if err == nil && len(jobErrStrs) == 0 { - break - } - - if i == retryTimes-1 { - logutil.BgLogger().Error("pause all jobs failed", zap.String("category", "upgrading"), zap.Strings("errs", jobErrStrs), zap.Error(err)) - return err - } - logutil.BgLogger().Warn("pause all jobs failed", zap.String("category", "upgrading"), zap.Strings("errs", jobErrStrs), zap.Error(err)) - time.Sleep(interval) - } logutil.BgLogger().Info("update global state to upgrading", zap.String("category", "upgrading"), zap.String("state", syncer.StateUpgrading)) return nil } diff --git a/session/bootstraptest/bootstrap_upgrade_test.go b/session/bootstraptest/bootstrap_upgrade_test.go index 760342d01edd9..42fd20db59ca8 100644 --- a/session/bootstraptest/bootstrap_upgrade_test.go +++ b/session/bootstraptest/bootstrap_upgrade_test.go @@ -554,6 +554,10 @@ func execute(ctx context.Context, s sessionctx.Context, query string) ([]chunk.R return rows, nil } +// TestUpgradeWithPauseDDL adds a user and a system DB's DDL operations, before every test bootstrap(DDL operation). It tests: +// +// 1.Before and after each test bootstrap, the DDL of the user DB is paused, but the DDL of the system DB is not paused. +// 2.Check user DDLs are handled after system DDLs. func TestUpgradeWithPauseDDL(t *testing.T) { session.SupportUpgradeStateVer-- ddl.SetWaitTimeWhenErrorOccurred(1 * time.Microsecond) @@ -564,6 +568,12 @@ func TestUpgradeWithPauseDDL(t *testing.T) { require.NoError(t, err) _, err = execute(context.Background(), se, "create table test.pause_user_ddl_t(a int, b int)") require.NoError(t, err) + _, err = execute(context.Background(), se, "create table mysql.pause_user_ddl_t(a int, b int)") + require.NoError(t, err) + _, err = execute(context.Background(), se, "create table test.pause_user_ddl_t1(a int, b int)") + require.NoError(t, err) + _, err = execute(context.Background(), se, "create table mysql.pause_user_ddl_t1(a int, b int)") + require.NoError(t, err) tc := session.TestCallback{Cnt: atomicutil.NewInt32(0)} sql := "select job_meta, processing from mysql.tidb_ddl_job where job_id in (select min(job_id) from mysql.tidb_ddl_job group by schema_ids, table_ids, processing) order by processing desc, job_id" @@ -574,16 +584,7 @@ func TestUpgradeWithPauseDDL(t *testing.T) { } wg := sync.WaitGroup{} - tc.OnBootstrapExported = func(s session.Session) { - var query string - switch tc.Cnt.Load() % 2 { - case 0: - query = fmt.Sprintf("alter table test.pause_user_ddl_t add index idx_%d(a)", tc.Cnt.Load()) - case 1: - query = fmt.Sprintf("alter table test.pause_user_ddl_t add column c_%d int", tc.Cnt.Load()) - } - tc.Cnt.Add(1) - + asyncExecDDL := func(query string) { ch := make(chan struct{}) wg.Add(1) go func() { @@ -598,6 +599,22 @@ func TestUpgradeWithPauseDDL(t *testing.T) { } }() <-ch + } + // Before every test bootstrap(DDL operation), we add a user and a system DB's DDL operations. + tc.OnBootstrapExported = func(s session.Session) { + var query1, query2 string + switch tc.Cnt.Load() % 2 { + case 0: + query1 = fmt.Sprintf("alter table mysql.pause_user_ddl_t add index idx_%d(a)", tc.Cnt.Load()) + query2 = fmt.Sprintf("alter table test.pause_user_ddl_t add column c_%d int", tc.Cnt.Load()) + case 1: + // Make sure case0 and case1 use different table ID. Then case1's table won't be filtered because they use the same table ID. + query1 = fmt.Sprintf("alter table test.pause_user_ddl_t1 add index idx_%d(a)", tc.Cnt.Load()) + query2 = fmt.Sprintf("alter table mysql.pause_user_ddl_t1 add column c_%d int", tc.Cnt.Load()) + } + tc.Cnt.Add(1) + asyncExecDDL(query1) + asyncExecDDL(query2) rows, err := execute(context.Background(), s, sql) require.NoError(t, err) From 7b5f48b227cb6cd1a4096e13d408d8005b15e1c3 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Mon, 21 Aug 2023 13:53:04 +0800 Subject: [PATCH 21/32] *: use std/slices to replace exp/slices (#46250) ref pingcap/tidb#45933 --- planner/util/BUILD.bazel | 1 - planner/util/path.go | 3 ++- statistics/debugtrace.go | 2 +- statistics/histogram.go | 12 ++++++------ testkit/BUILD.bazel | 1 - testkit/result.go | 13 +++---------- testkit/testutil/BUILD.bazel | 1 - testkit/testutil/handle.go | 2 +- timer/api/BUILD.bazel | 1 - timer/api/mem_store.go | 2 +- 10 files changed, 14 insertions(+), 24 deletions(-) diff --git a/planner/util/BUILD.bazel b/planner/util/BUILD.bazel index 50f4512126508..00fd783b89041 100644 --- a/planner/util/BUILD.bazel +++ b/planner/util/BUILD.bazel @@ -19,7 +19,6 @@ go_library( "//util/collate", "//util/ranger", "//util/size", - "@org_golang_x_exp//slices", ], ) diff --git a/planner/util/path.go b/planner/util/path.go index 68b11bbf5751f..7f959176aef61 100644 --- a/planner/util/path.go +++ b/planner/util/path.go @@ -15,6 +15,8 @@ package util import ( + "slices" + "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" @@ -23,7 +25,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/ranger" - "golang.org/x/exp/slices" ) // AccessPath indicates the way we access a table: by using single index, or by using multiple indexes, diff --git a/statistics/debugtrace.go b/statistics/debugtrace.go index 10d5c52cd2d4b..4bd855a719f3e 100644 --- a/statistics/debugtrace.go +++ b/statistics/debugtrace.go @@ -16,13 +16,13 @@ package statistics import ( "encoding/json" + "slices" "github.com/pingcap/tidb/planner/util/debugtrace" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" "golang.org/x/exp/maps" - "golang.org/x/exp/slices" ) /* diff --git a/statistics/histogram.go b/statistics/histogram.go index 7b334539414ee..2b757d087f784 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -18,6 +18,7 @@ import ( "bytes" "fmt" "math" + "slices" "sort" "strings" "time" @@ -42,7 +43,6 @@ import ( "github.com/pingcap/tipb/go-tipb" "github.com/twmb/murmur3" "go.uber.org/zap" - "golang.org/x/exp/slices" ) // Histogram represents statistics for a column or index. @@ -347,8 +347,8 @@ func (hg *Histogram) RemoveVals(valCntPairs []TopNMeta) { // AddIdxVals adds the given values to the histogram. func (hg *Histogram) AddIdxVals(idxValCntPairs []TopNMeta) { totalAddCnt := int64(0) - slices.SortFunc(idxValCntPairs, func(i, j TopNMeta) bool { - return bytes.Compare(i.Encoded, j.Encoded) < 0 + slices.SortFunc(idxValCntPairs, func(i, j TopNMeta) int { + return bytes.Compare(i.Encoded, j.Encoded) }) for bktIdx, pairIdx := 0, 0; bktIdx < hg.Len(); bktIdx++ { for pairIdx < len(idxValCntPairs) { @@ -1540,19 +1540,19 @@ func MergePartitionHist2GlobalHist(sc *stmtctx.StatementContext, hists []*Histog buckets = buckets[:tail] var sortError error - slices.SortFunc(buckets, func(i, j *bucket4Merging) bool { + slices.SortFunc(buckets, func(i, j *bucket4Merging) int { res, err := i.upper.Compare(sc, j.upper, collate.GetBinaryCollator()) if err != nil { sortError = err } if res != 0 { - return res < 0 + return res } res, err = i.lower.Compare(sc, j.lower, collate.GetBinaryCollator()) if err != nil { sortError = err } - return res < 0 + return res }) if sortError != nil { return nil, sortError diff --git a/testkit/BUILD.bazel b/testkit/BUILD.bazel index de75ed3c996c9..aca65b9c5b78d 100644 --- a/testkit/BUILD.bazel +++ b/testkit/BUILD.bazel @@ -44,7 +44,6 @@ go_library( "@com_github_tikv_client_go_v2//tikv", "@com_github_tikv_client_go_v2//tikvrpc", "@io_opencensus_go//stats/view", - "@org_golang_x_exp//slices", "@org_uber_go_atomic//:atomic", "@org_uber_go_zap//:zap", ], diff --git a/testkit/result.go b/testkit/result.go index 4995a42778b84..417455d3f0988 100644 --- a/testkit/result.go +++ b/testkit/result.go @@ -19,11 +19,11 @@ package testkit import ( "bytes" "fmt" + "slices" "strings" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "golang.org/x/exp/slices" ) // Result is the result returned by MustQuery. @@ -86,15 +86,8 @@ func Rows(args ...string) [][]interface{} { // Sort sorts and return the result. func (res *Result) Sort() *Result { - slices.SortFunc(res.rows, func(a, b []string) bool { - for i := range a { - if a[i] < b[i] { - return true - } else if a[i] > b[i] { - return false - } - } - return false + slices.SortFunc(res.rows, func(a, b []string) int { + return slices.Compare(a, b) }) return res } diff --git a/testkit/testutil/BUILD.bazel b/testkit/testutil/BUILD.bazel index 3f4ed7312c01c..25e0a5265f6d4 100644 --- a/testkit/testutil/BUILD.bazel +++ b/testkit/testutil/BUILD.bazel @@ -19,7 +19,6 @@ go_library( "//util/logutil", "@com_github_pingcap_log//:log", "@com_github_stretchr_testify//require", - "@org_golang_x_exp//slices", "@org_uber_go_zap//:zap", "@org_uber_go_zap//zapcore", ], diff --git a/testkit/testutil/handle.go b/testkit/testutil/handle.go index 52161acbd1112..c0832562f8816 100644 --- a/testkit/testutil/handle.go +++ b/testkit/testutil/handle.go @@ -17,6 +17,7 @@ package testutil import ( + "slices" "testing" "github.com/pingcap/tidb/kv" @@ -25,7 +26,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/stretchr/testify/require" - "golang.org/x/exp/slices" ) // MustNewCommonHandle create a common handle with given values. diff --git a/timer/api/BUILD.bazel b/timer/api/BUILD.bazel index 5419f6d33e3dd..4e42f9168e1a4 100644 --- a/timer/api/BUILD.bazel +++ b/timer/api/BUILD.bazel @@ -18,7 +18,6 @@ go_library( "//util/logutil", "@com_github_google_uuid//:uuid", "@com_github_pingcap_errors//:errors", - "@org_golang_x_exp//slices", "@org_uber_go_zap//:zap", ], ) diff --git a/timer/api/mem_store.go b/timer/api/mem_store.go index 63e16d1639ded..e4e884fc7ea3d 100644 --- a/timer/api/mem_store.go +++ b/timer/api/mem_store.go @@ -17,12 +17,12 @@ package api import ( "context" "encoding/hex" + "slices" "sync" "time" "github.com/google/uuid" "github.com/pingcap/errors" - "golang.org/x/exp/slices" ) type memStoreWatcher struct { From bc88e13edd8d4857fc1171aa1da3ec815d3e5e7d Mon Sep 17 00:00:00 2001 From: tangenta Date: Mon, 21 Aug 2023 14:39:33 +0800 Subject: [PATCH 22/32] workerpool: generic result type for worker pool (#46185) ref pingcap/tidb#46258 --- ddl/backfilling_scheduler.go | 11 ++- executor/executor.go | 8 +- resourcemanager/pool/workerpool/BUILD.bazel | 3 +- resourcemanager/pool/workerpool/workerpool.go | 98 +++++++++++-------- .../pool/workerpool/workpool_test.go | 96 +++++++++++++++++- 5 files changed, 162 insertions(+), 54 deletions(-) diff --git a/ddl/backfilling_scheduler.go b/ddl/backfilling_scheduler.go index caa1e96cfc15b..6596b3e49d611 100644 --- a/ddl/backfilling_scheduler.go +++ b/ddl/backfilling_scheduler.go @@ -268,7 +268,7 @@ type ingestBackfillScheduler struct { copReqSenderPool *copReqSenderPool - writerPool *workerpool.WorkerPool[idxRecResult] + writerPool *workerpool.WorkerPool[idxRecResult, workerpool.None] writerMaxID int poolErr chan error backendCtx ingest.BackendCtx @@ -308,12 +308,12 @@ func (b *ingestBackfillScheduler) setupWorkers() error { } b.copReqSenderPool = copReqSenderPool readerCnt, writerCnt := b.expectedWorkerSize() - skipReg := workerpool.OptionSkipRegister[idxRecResult]{} writerPool, err := workerpool.NewWorkerPool[idxRecResult]("ingest_writer", - poolutil.DDL, writerCnt, b.createWorker, skipReg) + poolutil.DDL, writerCnt, b.createWorker) if err != nil { return errors.Trace(err) } + writerPool.Start() b.writerPool = writerPool b.copReqSenderPool.chunkSender = writerPool b.copReqSenderPool.adjustSize(readerCnt) @@ -382,7 +382,7 @@ func (b *ingestBackfillScheduler) adjustWorkerSize() error { return nil } -func (b *ingestBackfillScheduler) createWorker() workerpool.Worker[idxRecResult] { +func (b *ingestBackfillScheduler) createWorker() workerpool.Worker[idxRecResult, workerpool.None] { reorgInfo := b.reorgInfo job := reorgInfo.Job sessCtx, err := newSessCtx(reorgInfo) @@ -447,7 +447,7 @@ func (*ingestBackfillScheduler) expectedWorkerSize() (readerSize int, writerSize return readerSize, writerSize } -func (w *addIndexIngestWorker) HandleTask(rs idxRecResult) { +func (w *addIndexIngestWorker) HandleTask(rs idxRecResult) (_ workerpool.None) { defer util.Recover(metrics.LabelDDL, "ingestWorker.HandleTask", func() { w.resultCh <- &backfillResult{taskID: rs.id, err: dbterror.ErrReorgPanic} }, false) @@ -494,6 +494,7 @@ func (w *addIndexIngestWorker) HandleTask(rs idxRecResult) { ResultCounterForTest.Add(1) } w.resultCh <- result + return } func (*addIndexIngestWorker) Close() {} diff --git a/executor/executor.go b/executor/executor.go index e6856a6703193..f7eb3dabb1a43 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -2350,7 +2350,7 @@ func getCheckSum(ctx context.Context, se sessionctx.Context, sql string) ([]grou } // HandleTask implements the Worker interface. -func (w *checkIndexWorker) HandleTask(task checkIndexTask) { +func (w *checkIndexWorker) HandleTask(task checkIndexTask) (_ workerpool.None) { defer w.e.wg.Done() idxInfo := w.indexInfos[task.indexOffset] bucketSize := int(CheckTableFastBucketSize.Load()) @@ -2688,12 +2688,13 @@ func (w *checkIndexWorker) HandleTask(task checkIndexTask) { } } } + return } // Close implements the Worker interface. func (*checkIndexWorker) Close() {} -func (e *FastCheckTableExec) createWorker() workerpool.Worker[checkIndexTask] { +func (e *FastCheckTableExec) createWorker() workerpool.Worker[checkIndexTask, workerpool.None] { return &checkIndexWorker{sctx: e.Ctx(), dbName: e.dbName, table: e.table, indexInfos: e.indexInfos, e: e} } @@ -2711,10 +2712,11 @@ func (e *FastCheckTableExec) Next(context.Context, *chunk.Chunk) error { }() workerPool, err := workerpool.NewWorkerPool[checkIndexTask]("checkIndex", - poolutil.CheckTable, 3, e.createWorker, workerpool.OptionSkipRegister[checkIndexTask]{}) + poolutil.CheckTable, 3, e.createWorker) if err != nil { return errors.Trace(err) } + workerPool.Start() e.wg.Add(len(e.indexInfos)) for i := range e.indexInfos { diff --git a/resourcemanager/pool/workerpool/BUILD.bazel b/resourcemanager/pool/workerpool/BUILD.bazel index cc6f0222e834b..30647736b656d 100644 --- a/resourcemanager/pool/workerpool/BUILD.bazel +++ b/resourcemanager/pool/workerpool/BUILD.bazel @@ -7,7 +7,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//metrics", - "//resourcemanager", "//resourcemanager/util", "//util", "//util/syncutil", @@ -25,11 +24,13 @@ go_test( embed = [":workerpool"], flaky = True, race = "on", + shard_count = 3, deps = [ "//resourcemanager/util", "//testkit/testsetup", "//util/logutil", "@com_github_stretchr_testify//require", + "@org_golang_x_sync//errgroup", "@org_uber_go_goleak//:goleak", "@org_uber_go_zap//:zap", ], diff --git a/resourcemanager/pool/workerpool/workerpool.go b/resourcemanager/pool/workerpool/workerpool.go index e97ed218ee228..dee92351b674d 100644 --- a/resourcemanager/pool/workerpool/workerpool.go +++ b/resourcemanager/pool/workerpool/workerpool.go @@ -18,7 +18,6 @@ import ( "time" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/resourcemanager" "github.com/pingcap/tidb/resourcemanager/util" tidbutil "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/syncutil" @@ -26,85 +25,99 @@ import ( ) // Worker is worker interface. -type Worker[T any] interface { - HandleTask(task T) +type Worker[T, R any] interface { + HandleTask(task T) R Close() } // WorkerPool is a pool of workers. -type WorkerPool[T any] struct { +type WorkerPool[T, R any] struct { name string numWorkers int32 originWorkers int32 runningTask atomicutil.Int32 taskChan chan T + resChan chan R quitChan chan struct{} wg tidbutil.WaitGroupWrapper - createWorker func() Worker[T] + createWorker func() Worker[T, R] lastTuneTs atomicutil.Time mu syncutil.RWMutex - skipRegister bool } // Option is the config option for WorkerPool. -type Option[T any] interface { - Apply(pool *WorkerPool[T]) +type Option[T, R any] interface { + Apply(pool *WorkerPool[T, R]) } -// OptionSkipRegister is an option to skip register the worker pool to resource manager. -type OptionSkipRegister[T any] struct{} - -// Apply implements the Option interface. -func (OptionSkipRegister[T]) Apply(pool *WorkerPool[T]) { - pool.skipRegister = true -} +// None is a type placeholder for the worker pool that does not have a result receiver. +type None struct{} // NewWorkerPool creates a new worker pool. -func NewWorkerPool[T any](name string, component util.Component, numWorkers int, - createWorker func() Worker[T], opts ...Option[T]) (*WorkerPool[T], error) { +func NewWorkerPool[T, R any](name string, _ util.Component, numWorkers int, + createWorker func() Worker[T, R], opts ...Option[T, R]) (*WorkerPool[T, R], error) { if numWorkers <= 0 { numWorkers = 1 } - p := &WorkerPool[T]{ + p := &WorkerPool[T, R]{ name: name, numWorkers: int32(numWorkers), originWorkers: int32(numWorkers), - taskChan: make(chan T), quitChan: make(chan struct{}), - createWorker: createWorker, } for _, opt := range opts { opt.Apply(p) } - if !p.skipRegister { - err := resourcemanager.InstanceResourceManager.Register(p, name, component) - if err != nil { - return nil, err + p.createWorker = createWorker + return p, nil +} + +// SetTaskReceiver sets the task receiver for the pool. +func (p *WorkerPool[T, R]) SetTaskReceiver(recv chan T) { + p.taskChan = recv +} + +// SetResultSender sets the result sender for the pool. +func (p *WorkerPool[T, R]) SetResultSender(sender chan R) { + p.resChan = sender +} + +// Start starts default count of workers. +func (p *WorkerPool[T, R]) Start() { + if p.taskChan == nil { + p.taskChan = make(chan T) + } + + if p.resChan == nil { + var zero R + var r interface{} = zero + if _, ok := r.(None); !ok { + p.resChan = make(chan R) } } - // Start default count of workers. for i := 0; i < int(p.numWorkers); i++ { p.runAWorker() } - - return p, nil } -func (p *WorkerPool[T]) handleTaskWithRecover(w Worker[T], task T) { +func (p *WorkerPool[T, R]) handleTaskWithRecover(w Worker[T, R], task T) { p.runningTask.Add(1) defer func() { p.runningTask.Add(-1) }() defer tidbutil.Recover(metrics.LabelWorkerPool, "handleTaskWithRecover", nil, false) - w.HandleTask(task) + r := w.HandleTask(task) + if p.resChan != nil { + p.resChan <- r + } } -func (p *WorkerPool[T]) runAWorker() { +func (p *WorkerPool[T, R]) runAWorker() { w := p.createWorker() if w == nil { return // Fail to create worker, quit. @@ -123,12 +136,17 @@ func (p *WorkerPool[T]) runAWorker() { } // AddTask adds a task to the pool. -func (p *WorkerPool[T]) AddTask(task T) { +func (p *WorkerPool[T, R]) AddTask(task T) { p.taskChan <- task } +// GetResultChan gets the result channel from the pool. +func (p *WorkerPool[T, R]) GetResultChan() <-chan R { + return p.resChan +} + // Tune tunes the pool to the specified number of workers. -func (p *WorkerPool[T]) Tune(numWorkers int32) { +func (p *WorkerPool[T, R]) Tune(numWorkers int32) { if numWorkers <= 0 { numWorkers = 1 } @@ -151,37 +169,37 @@ func (p *WorkerPool[T]) Tune(numWorkers int32) { } // LastTunerTs returns the last time when the pool was tuned. -func (p *WorkerPool[T]) LastTunerTs() time.Time { +func (p *WorkerPool[T, R]) LastTunerTs() time.Time { return p.lastTuneTs.Load() } // Cap returns the capacity of the pool. -func (p *WorkerPool[T]) Cap() int32 { +func (p *WorkerPool[T, R]) Cap() int32 { p.mu.RLock() defer p.mu.RUnlock() return p.numWorkers } // Running returns the number of running workers. -func (p *WorkerPool[T]) Running() int32 { +func (p *WorkerPool[T, R]) Running() int32 { return p.runningTask.Load() } // Name returns the name of the pool. -func (p *WorkerPool[T]) Name() string { +func (p *WorkerPool[T, R]) Name() string { return p.name } // ReleaseAndWait releases the pool and wait for complete. -func (p *WorkerPool[T]) ReleaseAndWait() { +func (p *WorkerPool[T, R]) ReleaseAndWait() { close(p.quitChan) p.wg.Wait() - if !p.skipRegister { - resourcemanager.InstanceResourceManager.Unregister(p.Name()) + if p.resChan != nil { + close(p.resChan) } } // GetOriginConcurrency return the concurrency of the pool at the init. -func (p *WorkerPool[T]) GetOriginConcurrency() int32 { +func (p *WorkerPool[T, R]) GetOriginConcurrency() int32 { return p.originWorkers } diff --git a/resourcemanager/pool/workerpool/workpool_test.go b/resourcemanager/pool/workerpool/workpool_test.go index 21e800706820e..0d21795603d30 100644 --- a/resourcemanager/pool/workerpool/workpool_test.go +++ b/resourcemanager/pool/workerpool/workpool_test.go @@ -23,35 +23,48 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/stretchr/testify/require" "go.uber.org/zap" + "golang.org/x/sync/errgroup" ) var globalCnt atomic.Int64 var cntWg sync.WaitGroup -type MyWorker[T int64] struct { +type MyWorker[T int64, R struct{}] struct { id int } -func (w *MyWorker[T]) HandleTask(task int64) { +func (w *MyWorker[T, R]) HandleTask(task int64) struct{} { globalCnt.Add(task) cntWg.Done() logutil.BgLogger().Info("Worker handling task") + return struct{}{} } -func (w *MyWorker[T]) Close() { +func (w *MyWorker[T, R]) Close() { logutil.BgLogger().Info("Close worker", zap.Any("id", w.id)) } -func createMyWorker() Worker[int64] { - return &MyWorker[int64]{} +func createMyWorker() Worker[int64, struct{}] { + return &MyWorker[int64, struct{}]{} } func TestWorkerPool(t *testing.T) { // Create a worker pool with 3 workers. pool, err := NewWorkerPool[int64]("test", util.UNKNOWN, 3, createMyWorker) require.NoError(t, err) + pool.Start() globalCnt.Store(0) + g := new(errgroup.Group) + g.Go(func() error { + // Consume the results. + for range pool.GetResultChan() { + // Do nothing. + } + return nil + }) + defer g.Wait() + // Add some tasks to the pool. cntWg.Add(10) for i := 0; i < 10; i++ { @@ -91,3 +104,76 @@ func TestWorkerPool(t *testing.T) { // Wait for the tasks to be completed. pool.ReleaseAndWait() } + +type dummyWorker[T, R any] struct { +} + +func (d dummyWorker[T, R]) HandleTask(task T) R { + var zero R + return zero +} + +func (d dummyWorker[T, R]) Close() {} + +func TestWorkerPoolNoneResult(t *testing.T) { + pool, err := NewWorkerPool[int64, None]( + "test", util.UNKNOWN, 3, + func() Worker[int64, None] { + return dummyWorker[int64, None]{} + }) + require.NoError(t, err) + pool.Start() + ch := pool.GetResultChan() + require.Nil(t, ch) + pool.ReleaseAndWait() + + pool2, err := NewWorkerPool[int64, int64]( + "test", util.UNKNOWN, 3, + func() Worker[int64, int64] { + return dummyWorker[int64, int64]{} + }) + require.NoError(t, err) + pool2.Start() + require.NotNil(t, pool2.GetResultChan()) + pool2.ReleaseAndWait() + + pool3, err := NewWorkerPool[int64, struct{}]( + "test", util.UNKNOWN, 3, + func() Worker[int64, struct{}] { + return dummyWorker[int64, struct{}]{} + }) + require.NoError(t, err) + pool3.Start() + require.NotNil(t, pool3.GetResultChan()) + pool3.ReleaseAndWait() +} + +func TestWorkerPoolCustomChan(t *testing.T) { + pool, err := NewWorkerPool[int64, int64]( + "test", util.UNKNOWN, 3, + func() Worker[int64, int64] { + return dummyWorker[int64, int64]{} + }) + require.NoError(t, err) + + taskCh := make(chan int64) + pool.SetTaskReceiver(taskCh) + resultCh := make(chan int64) + pool.SetResultSender(resultCh) + count := 0 + g := new(errgroup.Group) + g.Go(func() error { + for range resultCh { + count++ + } + return nil + }) + + pool.Start() + for i := 0; i < 5; i++ { + taskCh <- int64(i) + } + pool.ReleaseAndWait() + require.NoError(t, g.Wait()) + require.Equal(t, 5, count) +} From 533998e5921a8f662c878b60a5d0c9608d52d736 Mon Sep 17 00:00:00 2001 From: Hangjie Mo Date: Mon, 21 Aug 2023 15:18:33 +0800 Subject: [PATCH 23/32] executor: fix retChk = nil in index_merge_reader (#46111) close pingcap/tidb#46005 --- executor/index_merge_reader.go | 42 ++++++++++--------- executor/test/indexmergereadtest/BUILD.bazel | 2 +- .../index_merge_reader_test.go | 13 ++++++ 3 files changed, 36 insertions(+), 21 deletions(-) diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index a2e0d8db90d48..faa49266d8628 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -665,7 +665,11 @@ func (w *partialTableWorker) getRetTpsForTableScan() []*types.FieldType { func (w *partialTableWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, handleCols plannercore.HandleCols) ( handles []kv.Handle, retChk *chunk.Chunk, err error) { handles = make([]kv.Handle, 0, w.batchSize) + if len(w.byItems) != 0 { + retChk = chunk.NewChunkWithCapacity(w.getRetTpsForTableScan(), w.batchSize) + } var memUsage int64 + var chunkRowOffset int defer w.memTracker.Consume(-memUsage) for len(handles) < w.batchSize { requiredRows := w.batchSize - len(handles) @@ -679,7 +683,7 @@ func (w *partialTableWorker) extractTaskHandles(ctx context.Context, chk *chunk. start := time.Now() err = errors.Trace(w.tableReader.Next(ctx, chk)) if err != nil { - return handles, nil, err + return nil, nil, err } if be := w.tableReader.Base(); be != nil && be.RuntimeStats() != nil { be.RuntimeStats().Record(time.Since(start), chk.NumRows()) @@ -693,12 +697,12 @@ func (w *partialTableWorker) extractTaskHandles(ctx context.Context, chk *chunk. memDelta := chk.MemoryUsage() memUsage += memDelta w.memTracker.Consume(memDelta) - for i := 0; i < chk.NumRows(); i++ { + for chunkRowOffset = 0; chunkRowOffset < chk.NumRows(); chunkRowOffset++ { if w.pushedLimit != nil { w.scannedKeys++ if w.scannedKeys > (w.pushedLimit.Offset + w.pushedLimit.Count) { // Skip the handles after Offset+Count. - return handles, retChk, nil + break } } var handle kv.Handle @@ -707,21 +711,18 @@ func (w *partialTableWorker) extractTaskHandles(ctx context.Context, chk *chunk. return nil, nil, err1 } if ok { - handle, err = handleCols.BuildPartitionHandleFromIndexRow(chk.GetRow(i)) + handle, err = handleCols.BuildPartitionHandleFromIndexRow(chk.GetRow(chunkRowOffset)) } else { - handle, err = handleCols.BuildHandleFromIndexRow(chk.GetRow(i)) + handle, err = handleCols.BuildHandleFromIndexRow(chk.GetRow(chunkRowOffset)) } if err != nil { return nil, nil, err } handles = append(handles, handle) } - // used for limit embedded. + // used for order by if len(w.byItems) != 0 { - if retChk == nil { - retChk = chunk.NewChunkWithCapacity(w.getRetTpsForTableScan(), w.batchSize) - } - retChk.Append(chk, 0, chk.NumRows()) + retChk.Append(chk, 0, chunkRowOffset) } } w.batchSize *= 2 @@ -1595,7 +1596,11 @@ func (w *partialIndexWorker) getRetTpsForIndexScan(handleCols plannercore.Handle func (w *partialIndexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, idxResult distsql.SelectResult, handleCols plannercore.HandleCols) ( handles []kv.Handle, retChk *chunk.Chunk, err error) { handles = make([]kv.Handle, 0, w.batchSize) + if len(w.byItems) != 0 { + retChk = chunk.NewChunkWithCapacity(w.getRetTpsForIndexScan(handleCols), w.batchSize) + } var memUsage int64 + var chunkRowOffset int defer w.memTracker.Consume(-memUsage) for len(handles) < w.batchSize { requiredRows := w.batchSize - len(handles) @@ -1609,7 +1614,7 @@ func (w *partialIndexWorker) extractTaskHandles(ctx context.Context, chk *chunk. start := time.Now() err = errors.Trace(idxResult.Next(ctx, chk)) if err != nil { - return handles, nil, err + return nil, nil, err } if w.stats != nil && w.idxID != 0 { w.sc.GetSessionVars().StmtCtx.RuntimeStatsColl.GetBasicRuntimeStats(w.idxID).Record(time.Since(start), chk.NumRows()) @@ -1623,12 +1628,12 @@ func (w *partialIndexWorker) extractTaskHandles(ctx context.Context, chk *chunk. memDelta := chk.MemoryUsage() memUsage += memDelta w.memTracker.Consume(memDelta) - for i := 0; i < chk.NumRows(); i++ { + for chunkRowOffset = 0; chunkRowOffset < chk.NumRows(); chunkRowOffset++ { if w.pushedLimit != nil { w.scannedKeys++ if w.scannedKeys > (w.pushedLimit.Offset + w.pushedLimit.Count) { // Skip the handles after Offset+Count. - return handles, retChk, nil + break } } var handle kv.Handle @@ -1637,21 +1642,18 @@ func (w *partialIndexWorker) extractTaskHandles(ctx context.Context, chk *chunk. return nil, nil, err1 } if ok { - handle, err = handleCols.BuildPartitionHandleFromIndexRow(chk.GetRow(i)) + handle, err = handleCols.BuildPartitionHandleFromIndexRow(chk.GetRow(chunkRowOffset)) } else { - handle, err = handleCols.BuildHandleFromIndexRow(chk.GetRow(i)) + handle, err = handleCols.BuildHandleFromIndexRow(chk.GetRow(chunkRowOffset)) } if err != nil { return nil, nil, err } handles = append(handles, handle) } - // used for limit embedded. + // used for order by if len(w.byItems) != 0 { - if retChk == nil { - retChk = chunk.NewChunkWithCapacity(w.getRetTpsForIndexScan(handleCols), w.batchSize) - } - retChk.Append(chk, 0, chk.NumRows()) + retChk.Append(chk, 0, chunkRowOffset) } } w.batchSize *= 2 diff --git a/executor/test/indexmergereadtest/BUILD.bazel b/executor/test/indexmergereadtest/BUILD.bazel index b83f7a2a58dcc..558f8bc39c599 100644 --- a/executor/test/indexmergereadtest/BUILD.bazel +++ b/executor/test/indexmergereadtest/BUILD.bazel @@ -9,7 +9,7 @@ go_test( ], flaky = True, race = "on", - shard_count = 34, + shard_count = 35, deps = [ "//config", "//executor", diff --git a/executor/test/indexmergereadtest/index_merge_reader_test.go b/executor/test/indexmergereadtest/index_merge_reader_test.go index e6a37c19ab51e..ee7df9ed8bff6 100644 --- a/executor/test/indexmergereadtest/index_merge_reader_test.go +++ b/executor/test/indexmergereadtest/index_merge_reader_test.go @@ -1263,3 +1263,16 @@ func TestIndexMergeKeepOrderDirtyRead(t *testing.T) { tk.MustQuery(querySQL).Check(testkit.Rows("1 2 4", "1 1 1")) tk.MustExec("rollback") } + +func TestIssues46005(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_index_lookup_size = 1024") + tk.MustExec("create table t(a int, b int, c int, index idx1(a, c), index idx2(b, c))") + for i := 0; i < 1500; i++ { + tk.MustExec(fmt.Sprintf("insert into t(a,b,c) values (1, 1, %d)", i)) + } + + tk.MustQuery("select /*+ USE_INDEX_MERGE(t, idx1, idx2) */ * from t where a = 1 or b = 1 order by c limit 1025") +} From 94cfa8b0713ae606ed5811bed557b293461ed408 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 21 Aug 2023 16:18:34 +0800 Subject: [PATCH 24/32] planner: output a warning if plan rebuilding fails when reusing a cached plan (#46278) close pingcap/tidb#46159 --- planner/core/plan_cache.go | 2 +- planner/core/plan_cache_test.go | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index 8623f59173c12..13bdde829e48e 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -351,7 +351,7 @@ func RebuildPlan4CachedPlan(p Plan) (ok bool) { sc.InPreparedPlanBuilding = true defer func() { sc.InPreparedPlanBuilding = false }() if err := rebuildRange(p); err != nil { - // TODO: log or warn this error. + sc.AppendWarning(errors.Errorf("skip plan-cache: plan rebuild failed, %s", err.Error())) return false // fail to rebuild ranges } if !sc.UseCache { diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index b34700986e72d..dd92f46a30cdf 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -2446,6 +2446,18 @@ func TestIssue45378(t *testing.T) { tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) } +func TestIssue46159(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`create table t (a varchar(10), key(a(5)))`) + tk.MustExec(`prepare st from 'select a from t use index(a) where a=?'`) + tk.MustExec(`set @a='a'`) + tk.MustQuery(`execute st using @a`).Check(testkit.Rows()) + tk.MustQuery(`execute st using @a`).Check(testkit.Rows()) + tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 skip plan-cache: plan rebuild failed, rebuild to get an unsafe range")) +} + func TestBuiltinFuncFlen(t *testing.T) { // same as TestIssue45378 and TestIssue45253 store := testkit.CreateMockStore(t) From 9411e829e300c3c98523dccd67548820fb50d881 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Mon, 21 Aug 2023 17:35:05 +0800 Subject: [PATCH 25/32] statistics: use goroutine pool to improve performance (#46266) close pingcap/tidb#46267 --- go.mod | 2 +- statistics/BUILD.bazel | 1 + statistics/cmsketch_bench_test.go | 7 +++++-- statistics/handle/BUILD.bazel | 1 + statistics/handle/handle.go | 22 +++++++++++++++------- 5 files changed, 23 insertions(+), 10 deletions(-) diff --git a/go.mod b/go.mod index 6b0953dc30adf..85257648ee8c2 100644 --- a/go.mod +++ b/go.mod @@ -259,7 +259,7 @@ require ( github.com/spaolacci/murmur3 v1.1.0 // indirect github.com/spf13/cast v1.5.0 // indirect github.com/stretchr/objx v0.5.0 // indirect - github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect + github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a github.com/tklauser/go-sysconf v0.3.11 // indirect github.com/tklauser/numcpus v0.6.0 // indirect github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 // indirect diff --git a/statistics/BUILD.bazel b/statistics/BUILD.bazel index bd200e9610ab5..9fdfe41144e49 100644 --- a/statistics/BUILD.bazel +++ b/statistics/BUILD.bazel @@ -121,6 +121,7 @@ go_test( "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_log//:log", "@com_github_stretchr_testify//require", + "@com_github_tiancaiamao_gp//:gp", "@org_uber_go_goleak//:goleak", "@org_uber_go_zap//:zap", ], diff --git a/statistics/cmsketch_bench_test.go b/statistics/cmsketch_bench_test.go index e68d102ca57da..9c6f2fdda6e87 100644 --- a/statistics/cmsketch_bench_test.go +++ b/statistics/cmsketch_bench_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/stretchr/testify/require" + "github.com/tiancaiamao/gp" ) // cmd: go test -run=^$ -bench=BenchmarkMergePartTopN2GlobalTopNWithHists -benchmem github.com/pingcap/tidb/statistics @@ -131,15 +132,17 @@ func benchmarkMergeGlobalStatsTopNByConcurrencyWithHists(partitions int, b *test } else if batchSize > handle.MaxPartitionMergeBatchSize { batchSize = handle.MaxPartitionMergeBatchSize } + gpool := gp.New(mergeConcurrency, 5*time.Minute) + defer gpool.Close() b.ResetTimer() for i := 0; i < b.N; i++ { // Benchmark merge 10 topN. - _, _, _, _ = handle.MergeGlobalStatsTopNByConcurrency(mergeConcurrency, batchSize, wrapper, loc, version, 10, false, &isKilled) + _, _, _, _ = handle.MergeGlobalStatsTopNByConcurrency(gpool, mergeConcurrency, batchSize, wrapper, loc, version, 10, false, &isKilled) } } var benchmarkSizes = []int{100, 1000, 10000, 100000, 1000000, 10000000} -var benchmarkConcurrencySizes = []int{100, 1000, 10000, 100000, 1000000, 10000000, 100000000} +var benchmarkConcurrencySizes = []int{100, 1000, 10000, 100000} func BenchmarkMergePartTopN2GlobalTopNWithHists(b *testing.B) { for _, size := range benchmarkSizes { diff --git a/statistics/handle/BUILD.bazel b/statistics/handle/BUILD.bazel index 0aaddbdfac8fa..651703064b9e8 100644 --- a/statistics/handle/BUILD.bazel +++ b/statistics/handle/BUILD.bazel @@ -48,6 +48,7 @@ go_library( "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_log//:log", "@com_github_pingcap_tipb//go-tipb", + "@com_github_tiancaiamao_gp//:gp", "@com_github_tikv_client_go_v2//oracle", "@org_uber_go_atomic//:atomic", "@org_uber_go_zap//:zap", diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 17402392fb548..dbc665c72d098 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -18,6 +18,7 @@ import ( "context" "encoding/json" "fmt" + "math" "slices" "strconv" "strings" @@ -41,12 +42,12 @@ import ( handle_metrics "github.com/pingcap/tidb/statistics/handle/metrics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/syncutil" + "github.com/tiancaiamao/gp" "github.com/tikv/client-go/v2/oracle" atomic2 "go.uber.org/atomic" "go.uber.org/zap" @@ -62,6 +63,9 @@ const ( // Handle can update stats info periodically. type Handle struct { + // this gpool is used to reuse goroutine in the mergeGlobalStatsTopN. + gpool *gp.Pool + pool sessionPool // initStatsCtx is the ctx only used for initStats @@ -483,6 +487,7 @@ type sessionPool interface { func NewHandle(ctx, initStatsCtx sessionctx.Context, lease time.Duration, pool sessionPool, tracker sessionctx.SysProcTracker, autoAnalyzeProcIDGetter func() uint64) (*Handle, error) { cfg := config.GetGlobalConfig() handle := &Handle{ + gpool: gp.New(math.MaxInt16, time.Minute), ddlEventCh: make(chan *ddlUtil.Event, 1000), listHead: &SessionStatsCollector{mapper: make(tableDeltaMap), rateMap: make(errorRateDeltaMap)}, idxUsageListHead: &SessionIndexUsageCollector{mapper: make(indexUsageMap)}, @@ -857,7 +862,7 @@ func (h *Handle) mergePartitionStats2GlobalStats(sc sessionctx.Context, // These remaining topN numbers will be used as a separate bucket for later histogram merging. var popedTopN []statistics.TopNMeta wrapper := statistics.NewStatsWrapper(allHg[i], allTopN[i]) - globalStats.TopN[i], popedTopN, allHg[i], err = mergeGlobalStatsTopN(sc, wrapper, sc.GetSessionVars().StmtCtx.TimeZone, sc.GetSessionVars().AnalyzeVersion, uint32(opts[ast.AnalyzeOptNumTopN]), isIndex == 1) + globalStats.TopN[i], popedTopN, allHg[i], err = mergeGlobalStatsTopN(h.gpool, sc, wrapper, sc.GetSessionVars().StmtCtx.TimeZone, sc.GetSessionVars().AnalyzeVersion, uint32(opts[ast.AnalyzeOptNumTopN]), isIndex == 1) if err != nil { return } @@ -889,7 +894,7 @@ func (h *Handle) mergePartitionStats2GlobalStats(sc sessionctx.Context, return } -func mergeGlobalStatsTopN(sc sessionctx.Context, wrapper *statistics.StatsWrapper, +func mergeGlobalStatsTopN(gp *gp.Pool, sc sessionctx.Context, wrapper *statistics.StatsWrapper, timeZone *time.Location, version int, n uint32, isIndex bool) (*statistics.TopN, []statistics.TopNMeta, []*statistics.Histogram, error) { mergeConcurrency := sc.GetSessionVars().AnalyzePartitionMergeConcurrency @@ -904,14 +909,14 @@ func mergeGlobalStatsTopN(sc sessionctx.Context, wrapper *statistics.StatsWrappe } else if batchSize > MaxPartitionMergeBatchSize { batchSize = MaxPartitionMergeBatchSize } - return MergeGlobalStatsTopNByConcurrency(mergeConcurrency, batchSize, wrapper, timeZone, version, n, isIndex, killed) + return MergeGlobalStatsTopNByConcurrency(gp, mergeConcurrency, batchSize, wrapper, timeZone, version, n, isIndex, killed) } // MergeGlobalStatsTopNByConcurrency merge partition topN by concurrency // To merge global stats topn by concurrency, we will separate the partition topn in concurrency part and deal it with different worker. // mergeConcurrency is used to control the total concurrency of the running worker, and mergeBatchSize is sued to control // the partition size for each worker to solve it -func MergeGlobalStatsTopNByConcurrency(mergeConcurrency, mergeBatchSize int, wrapper *statistics.StatsWrapper, +func MergeGlobalStatsTopNByConcurrency(gp *gp.Pool, mergeConcurrency, mergeBatchSize int, wrapper *statistics.StatsWrapper, timeZone *time.Location, version int, n uint32, isIndex bool, killed *uint32) (*statistics.TopN, []statistics.TopNMeta, []*statistics.Histogram, error) { if len(wrapper.AllTopN) < mergeConcurrency { @@ -927,13 +932,15 @@ func MergeGlobalStatsTopNByConcurrency(mergeConcurrency, mergeBatchSize int, wra tasks = append(tasks, task) start = end } - var wg util.WaitGroupWrapper + var wg sync.WaitGroup taskNum := len(tasks) taskCh := make(chan *statistics.TopnStatsMergeTask, taskNum) respCh := make(chan *statistics.TopnStatsMergeResponse, taskNum) for i := 0; i < mergeConcurrency; i++ { worker := statistics.NewTopnStatsMergeWorker(taskCh, respCh, wrapper, killed) - wg.Run(func() { + wg.Add(1) + gp.Go(func() { + defer wg.Done() worker.Run(timeZone, isIndex, n, version) }) } @@ -2322,4 +2329,5 @@ func (h *Handle) SetStatsCacheCapacity(c int64) { // Close stops the background func (h *Handle) Close() { h.statsCache.Load().Close() + h.gpool.Close() } From 2abf83ddd50755dc7c3af6d2f2cf669fab29051d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BA=8C=E6=89=8B=E6=8E=89=E5=8C=85=E5=B7=A5=E7=A8=8B?= =?UTF-8?q?=E5=B8=88?= Date: Mon, 21 Aug 2023 19:03:34 +0800 Subject: [PATCH 26/32] *: remove feedback (#45969) ref pingcap/tidb#46056 --- br/pkg/backup/client.go | 4 +- br/pkg/checksum/executor.go | 2 +- br/pkg/lightning/backend/local/duplicate.go | 4 +- ddl/index_cop.go | 3 +- ddl/reorg.go | 4 +- distsql/BUILD.bazel | 2 - distsql/distsql.go | 11 +- distsql/distsql_test.go | 7 +- distsql/request_builder.go | 112 +- distsql/request_builder_test.go | 30 +- distsql/select_result.go | 10 +- domain/domain.go | 19 - executor/admin.go | 9 +- executor/analyze_col.go | 2 +- executor/analyze_idx.go | 2 +- executor/builder.go | 84 +- executor/checksum.go | 2 +- executor/distsql.go | 44 +- executor/index_merge_reader.go | 29 +- executor/inspection_summary.go | 3 - executor/table_reader.go | 32 +- executor/table_readers_required_rows_test.go | 3 +- executor/test/analyzetest/analyze_test.go | 20 +- executor/test/executor/BUILD.bazel | 1 - executor/test/executor/executor_test.go | 18 - executor/test/issuetest/BUILD.bazel | 1 - .../test/issuetest/executor_issue_test.go | 18 - infoschema/metric_table_def.go | 25 - metrics/grafana/tidb.json | 423 ++---- metrics/metrics.go | 4 - metrics/stats.go | 60 +- planner/core/BUILD.bazel | 1 - planner/core/find_best_task.go | 25 - planner/core/fragment.go | 4 +- planner/core/physical_plans.go | 15 +- planner/core/planbuilder.go | 12 - planner/core/planbuilder_test.go | 6 - session/BUILD.bazel | 1 - session/bootstrap.go | 3 + session/session.go | 25 - sessionctx/context.go | 4 - sessionctx/variable/sysvar.go | 23 +- statistics/BUILD.bazel | 6 - statistics/feedback.go | 1076 --------------- statistics/feedback_test.go | 310 ----- statistics/handle/BUILD.bazel | 4 - statistics/handle/gc.go | 3 - statistics/handle/handle.go | 37 - .../handle/handletest/globalstats/BUILD.bazel | 4 +- .../globalstats/globalstats_test.go | 96 -- statistics/handle/historical_stats_handler.go | 2 - statistics/handle/update.go | 735 +--------- statistics/handle/update_list_test.go | 2 - statistics/handle/updatetest/BUILD.bazel | 6 +- statistics/handle/updatetest/update_test.go | 1204 ----------------- statistics/histogram.go | 7 +- types/datum.go | 1 - util/mock/context.go | 3 - 58 files changed, 226 insertions(+), 4377 deletions(-) delete mode 100644 statistics/feedback.go delete mode 100644 statistics/feedback_test.go diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index edbe3efd0ae98..ab52aac9ab692 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -471,7 +471,7 @@ func appendRanges(tbl *model.TableInfo, tblID int64) ([]kv.KeyRange, error) { } retRanges := make([]kv.KeyRange, 0, 1+len(tbl.Indices)) - kvRanges, err := distsql.TableHandleRangesToKVRanges(nil, []int64{tblID}, tbl.IsCommonHandle, ranges, nil) + kvRanges, err := distsql.TableHandleRangesToKVRanges(nil, []int64{tblID}, tbl.IsCommonHandle, ranges) if err != nil { return nil, errors.Trace(err) } @@ -482,7 +482,7 @@ func appendRanges(tbl *model.TableInfo, tblID int64) ([]kv.KeyRange, error) { continue } ranges = ranger.FullRange() - idxRanges, err := distsql.IndexRangesToKVRanges(nil, tblID, index.ID, ranges, nil) + idxRanges, err := distsql.IndexRangesToKVRanges(nil, tblID, index.ID, ranges) if err != nil { return nil, errors.Trace(err) } diff --git a/br/pkg/checksum/executor.go b/br/pkg/checksum/executor.go index a77b0df85fa93..896a7297bab83 100644 --- a/br/pkg/checksum/executor.go +++ b/br/pkg/checksum/executor.go @@ -238,7 +238,7 @@ func buildTableRequest( var builder distsql.RequestBuilder // Use low priority to reducing impact to other requests. builder.Request.Priority = kv.PriorityLow - return builder.SetHandleRanges(nil, tableID, tableInfo.IsCommonHandle, ranges, nil). + return builder.SetHandleRanges(nil, tableID, tableInfo.IsCommonHandle, ranges). SetStartTS(startTS). SetChecksumRequest(checksum). SetConcurrency(int(concurrency)). diff --git a/br/pkg/lightning/backend/local/duplicate.go b/br/pkg/lightning/backend/local/duplicate.go index 69c1a1feec8c0..721b685e12504 100644 --- a/br/pkg/lightning/backend/local/duplicate.go +++ b/br/pkg/lightning/backend/local/duplicate.go @@ -227,13 +227,13 @@ func tableHandleKeyRanges(tableInfo *model.TableInfo) (*tidbkv.KeyRanges, error) ranges = ranger.FullRange() } tableIDs := physicalTableIDs(tableInfo) - return distsql.TableHandleRangesToKVRanges(nil, tableIDs, tableInfo.IsCommonHandle, ranges, nil) + return distsql.TableHandleRangesToKVRanges(nil, tableIDs, tableInfo.IsCommonHandle, ranges) } // tableIndexKeyRanges returns all key ranges associated with the tableInfo and indexInfo. func tableIndexKeyRanges(tableInfo *model.TableInfo, indexInfo *model.IndexInfo) (*tidbkv.KeyRanges, error) { tableIDs := physicalTableIDs(tableInfo) - return distsql.IndexRangesToKVRangesForTables(nil, tableIDs, indexInfo.ID, ranger.FullRange(), nil) + return distsql.IndexRangesToKVRangesForTables(nil, tableIDs, indexInfo.ID, ranger.FullRange()) } // DupKVStream is a streaming interface for collecting duplicate key-value pairs. diff --git a/ddl/index_cop.go b/ddl/index_cop.go index 909981f70f015..d3fdd59fa5ad1 100644 --- a/ddl/index_cop.go +++ b/ddl/index_cop.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -429,7 +428,7 @@ func (c *copContext) buildTableScan(ctx context.Context, startTS uint64, start, if err != nil { return nil, err } - return distsql.Select(ctx, c.sessCtx, kvReq, c.fieldTps, statistics.NewQueryFeedback(0, nil, 0, false)) + return distsql.Select(ctx, c.sessCtx, kvReq, c.fieldTps) } func (c *copContext) fetchTableScanResult(ctx context.Context, result distsql.SelectResult, diff --git a/ddl/reorg.go b/ddl/reorg.go index 1abda7c828e47..4d82071f4ad3a 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -561,7 +561,7 @@ func (dc *ddlCtx) buildDescTableScan(ctx *JobContext, startTS uint64, tbl table. } else { ranges = ranger.FullIntRange(false) } - builder = b.SetHandleRanges(sctx.GetSessionVars().StmtCtx, tbl.GetPhysicalID(), tbl.Meta().IsCommonHandle, ranges, nil) + builder = b.SetHandleRanges(sctx.GetSessionVars().StmtCtx, tbl.GetPhysicalID(), tbl.Meta().IsCommonHandle, ranges) builder.SetDAGRequest(dagPB). SetStartTS(startTS). SetKeepOrder(true). @@ -578,7 +578,7 @@ func (dc *ddlCtx) buildDescTableScan(ctx *JobContext, startTS uint64, tbl table. return nil, errors.Trace(err) } - result, err := distsql.Select(ctx.ddlJobCtx, sctx, kvReq, getColumnsTypes(handleCols), statistics.NewQueryFeedback(0, nil, 0, false)) + result, err := distsql.Select(ctx.ddlJobCtx, sctx, kvReq, getColumnsTypes(handleCols)) if err != nil { return nil, errors.Trace(err) } diff --git a/distsql/BUILD.bazel b/distsql/BUILD.bazel index 3a4044139c8eb..2601df8d0d0ce 100644 --- a/distsql/BUILD.bazel +++ b/distsql/BUILD.bazel @@ -23,7 +23,6 @@ go_library( "//sessionctx", "//sessionctx/stmtctx", "//sessionctx/variable", - "//statistics", "//store/copr", "//tablecodec", "//telemetry", @@ -74,7 +73,6 @@ go_test( "//sessionctx", "//sessionctx/stmtctx", "//sessionctx/variable", - "//statistics", "//store/copr", "//tablecodec", "//testkit/testsetup", diff --git a/distsql/distsql.go b/distsql/distsql.go index a648732676d62..053f9ee9f8fe3 100644 --- a/distsql/distsql.go +++ b/distsql/distsql.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/tracing" @@ -46,7 +45,6 @@ func GenSelectResultFromResponse(sctx sessionctx.Context, fieldTypes []*types.Fi rowLen: len(fieldTypes), fieldTypes: fieldTypes, ctx: sctx, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), copPlanIDs: planIDs, rootPlanID: rootID, storeType: kv.TiFlash, @@ -55,7 +53,7 @@ func GenSelectResultFromResponse(sctx sessionctx.Context, fieldTypes []*types.Fi // Select sends a DAG request, returns SelectResult. // In kvReq, KeyRanges is required, Concurrency/KeepOrder/Desc/IsolationLevel/Priority are optional. -func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fieldTypes []*types.FieldType, fb *statistics.QueryFeedback) (SelectResult, error) { +func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fieldTypes []*types.FieldType) (SelectResult, error) { r, ctx := tracing.StartRegionEx(ctx, "distsql.Select") defer r.End() @@ -109,7 +107,6 @@ func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fie rowLen: len(fieldTypes), fieldTypes: fieldTypes, ctx: sctx, - feedback: fb, sqlType: label, memTracker: kvReq.MemTracker, storeType: kvReq.StoreType, @@ -144,8 +141,8 @@ func SetTiFlashConfVarsInContext(ctx context.Context, sctx sessionctx.Context) c // The difference from Select is that SelectWithRuntimeStats will set copPlanIDs into selectResult, // which can help selectResult to collect runtime stats. func SelectWithRuntimeStats(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, - fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []int, rootPlanID int) (SelectResult, error) { - sr, err := Select(ctx, sctx, kvReq, fieldTypes, fb) + fieldTypes []*types.FieldType, copPlanIDs []int, rootPlanID int) (SelectResult, error) { + sr, err := Select(ctx, sctx, kvReq, fieldTypes) if err != nil { return nil, err } @@ -173,7 +170,6 @@ func Analyze(ctx context.Context, client kv.Client, kvReq *kv.Request, vars inte result := &selectResult{ label: "analyze", resp: resp, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), sqlType: label, storeType: kvReq.StoreType, } @@ -191,7 +187,6 @@ func Checksum(ctx context.Context, client kv.Client, kvReq *kv.Request, vars int result := &selectResult{ label: "checksum", resp: resp, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), sqlType: metrics.LblGeneral, storeType: kvReq.StoreType, } diff --git a/distsql/distsql_test.go b/distsql/distsql_test.go index 9fbee11a07a74..49ed70c2b672d 100644 --- a/distsql/distsql_test.go +++ b/distsql/distsql_test.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" @@ -338,7 +337,7 @@ func createSelectNormalByBenchmarkTest(batch, totalRows int, ctx sessionctx.Cont // Test Next. var response SelectResult - response, _ = Select(context.TODO(), ctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false)) + response, _ = Select(context.TODO(), ctx, request, colTypes) result, _ := response.(*selectResult) resp, _ := result.resp.(*mockResponse) @@ -413,9 +412,9 @@ func createSelectNormal(t *testing.T, batch, totalRows int, planIDs []int, sctx // Test Next. var response SelectResult if planIDs == nil { - response, err = Select(context.TODO(), sctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false)) + response, err = Select(context.TODO(), sctx, request, colTypes) } else { - response, err = SelectWithRuntimeStats(context.TODO(), sctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false), planIDs, 1) + response, err = SelectWithRuntimeStats(context.TODO(), sctx, request, colTypes, planIDs, 1) } require.NoError(t, err) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 0d7a5912c1c0d..d165edc8edc74 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" @@ -86,9 +85,9 @@ func (builder *RequestBuilder) SetMemTracker(tracker *memory.Tracker) *RequestBu // to "KeyRanges" firstly. // Note this function should be deleted or at least not exported, but currently // br refers it, so have to keep it. -func (builder *RequestBuilder) SetTableRanges(tid int64, tableRanges []*ranger.Range, fb *statistics.QueryFeedback) *RequestBuilder { +func (builder *RequestBuilder) SetTableRanges(tid int64, tableRanges []*ranger.Range) *RequestBuilder { if builder.err == nil { - builder.Request.KeyRanges = kv.NewNonParitionedKeyRanges(TableRangesToKVRanges(tid, tableRanges, fb)) + builder.Request.KeyRanges = kv.NewNonParitionedKeyRanges(TableRangesToKVRanges(tid, tableRanges)) } return builder } @@ -97,7 +96,7 @@ func (builder *RequestBuilder) SetTableRanges(tid int64, tableRanges []*ranger.R // "ranges" to "KeyRanges" firstly. func (builder *RequestBuilder) SetIndexRanges(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range) *RequestBuilder { if builder.err == nil { - builder.Request.KeyRanges, builder.err = IndexRangesToKVRanges(sc, tid, idxID, ranges, nil) + builder.Request.KeyRanges, builder.err = IndexRangesToKVRanges(sc, tid, idxID, ranges) } return builder } @@ -106,24 +105,24 @@ func (builder *RequestBuilder) SetIndexRanges(sc *stmtctx.StatementContext, tid, // "ranges" to "KeyRanges" firstly. func (builder *RequestBuilder) SetIndexRangesForTables(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range) *RequestBuilder { if builder.err == nil { - builder.Request.KeyRanges, builder.err = IndexRangesToKVRangesForTables(sc, tids, idxID, ranges, nil) + builder.Request.KeyRanges, builder.err = IndexRangesToKVRangesForTables(sc, tids, idxID, ranges) } return builder } // SetHandleRanges sets "KeyRanges" for "kv.Request" by converting table handle range // "ranges" to "KeyRanges" firstly. -func (builder *RequestBuilder) SetHandleRanges(sc *stmtctx.StatementContext, tid int64, isCommonHandle bool, ranges []*ranger.Range, fb *statistics.QueryFeedback) *RequestBuilder { - builder = builder.SetHandleRangesForTables(sc, []int64{tid}, isCommonHandle, ranges, fb) +func (builder *RequestBuilder) SetHandleRanges(sc *stmtctx.StatementContext, tid int64, isCommonHandle bool, ranges []*ranger.Range) *RequestBuilder { + builder = builder.SetHandleRangesForTables(sc, []int64{tid}, isCommonHandle, ranges) builder.err = builder.Request.KeyRanges.SetToNonPartitioned() return builder } // SetHandleRangesForTables sets "KeyRanges" for "kv.Request" by converting table handle range // "ranges" to "KeyRanges" firstly for multiple tables. -func (builder *RequestBuilder) SetHandleRangesForTables(sc *stmtctx.StatementContext, tid []int64, isCommonHandle bool, ranges []*ranger.Range, fb *statistics.QueryFeedback) *RequestBuilder { +func (builder *RequestBuilder) SetHandleRangesForTables(sc *stmtctx.StatementContext, tid []int64, isCommonHandle bool, ranges []*ranger.Range) *RequestBuilder { if builder.err == nil { - builder.Request.KeyRanges, builder.err = TableHandleRangesToKVRanges(sc, tid, isCommonHandle, ranges, fb) + builder.Request.KeyRanges, builder.err = TableHandleRangesToKVRanges(sc, tid, isCommonHandle, ranges) } return builder } @@ -435,9 +434,9 @@ func (builder *RequestBuilder) SetConnID(connID uint64) *RequestBuilder { } // TableHandleRangesToKVRanges convert table handle ranges to "KeyRanges" for multiple tables. -func TableHandleRangesToKVRanges(sc *stmtctx.StatementContext, tid []int64, isCommonHandle bool, ranges []*ranger.Range, fb *statistics.QueryFeedback) (*kv.KeyRanges, error) { +func TableHandleRangesToKVRanges(sc *stmtctx.StatementContext, tid []int64, isCommonHandle bool, ranges []*ranger.Range) (*kv.KeyRanges, error) { if !isCommonHandle { - return tablesRangesToKVRanges(tid, ranges, fb), nil + return tablesRangesToKVRanges(tid, ranges), nil } return CommonHandleRangesToKVRanges(sc, tid, ranges) } @@ -445,45 +444,16 @@ func TableHandleRangesToKVRanges(sc *stmtctx.StatementContext, tid []int64, isCo // TableRangesToKVRanges converts table ranges to "KeyRange". // Note this function should not be exported, but currently // br refers to it, so have to keep it. -func TableRangesToKVRanges(tid int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) []kv.KeyRange { +func TableRangesToKVRanges(tid int64, ranges []*ranger.Range) []kv.KeyRange { if len(ranges) == 0 { return []kv.KeyRange{} } - return tablesRangesToKVRanges([]int64{tid}, ranges, fb).FirstPartitionRange() + return tablesRangesToKVRanges([]int64{tid}, ranges).FirstPartitionRange() } // tablesRangesToKVRanges converts table ranges to "KeyRange". -func tablesRangesToKVRanges(tids []int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) *kv.KeyRanges { - if fb == nil || fb.Hist == nil { - return tableRangesToKVRangesWithoutSplit(tids, ranges) - } - // The following codes are deprecated since the feedback is deprecated. - krs := make([]kv.KeyRange, 0, len(ranges)) - feedbackRanges := make([]*ranger.Range, 0, len(ranges)) - for _, ran := range ranges { - low := codec.EncodeInt(nil, ran.LowVal[0].GetInt64()) - high := codec.EncodeInt(nil, ran.HighVal[0].GetInt64()) - if ran.LowExclude { - low = kv.Key(low).PrefixNext() - } - // If this range is split by histogram, then the high val will equal to one bucket's upper bound, - // since we need to guarantee each range falls inside the exactly one bucket, `PrefixNext` will make the - // high value greater than upper bound, so we store the range here. - r := &ranger.Range{LowVal: []types.Datum{types.NewBytesDatum(low)}, - HighVal: []types.Datum{types.NewBytesDatum(high)}, Collators: collate.GetBinaryCollatorSlice(1)} - feedbackRanges = append(feedbackRanges, r) - - if !ran.HighExclude { - high = kv.Key(high).PrefixNext() - } - for _, tid := range tids { - startKey := tablecodec.EncodeRowKey(tid, low) - endKey := tablecodec.EncodeRowKey(tid, high) - krs = append(krs, kv.KeyRange{StartKey: startKey, EndKey: endKey}) - } - } - fb.StoreRanges(feedbackRanges) - return kv.NewNonParitionedKeyRanges(krs) +func tablesRangesToKVRanges(tids []int64, ranges []*ranger.Range) *kv.KeyRanges { + return tableRangesToKVRangesWithoutSplit(tids, ranges) } func tableRangesToKVRangesWithoutSplit(tids []int64, ranges []*ranger.Range) *kv.KeyRanges { @@ -656,14 +626,14 @@ func PartitionHandlesToKVRanges(handles []kv.Handle) ([]kv.KeyRange, []int) { } // IndexRangesToKVRanges converts index ranges to "KeyRange". -func IndexRangesToKVRanges(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) (*kv.KeyRanges, error) { - return IndexRangesToKVRangesWithInterruptSignal(sc, tid, idxID, ranges, fb, nil, nil) +func IndexRangesToKVRanges(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range) (*kv.KeyRanges, error) { + return IndexRangesToKVRangesWithInterruptSignal(sc, tid, idxID, ranges, nil, nil) } // IndexRangesToKVRangesWithInterruptSignal converts index ranges to "KeyRange". // The process can be interrupted by set `interruptSignal` to true. -func IndexRangesToKVRangesWithInterruptSignal(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback, memTracker *memory.Tracker, interruptSignal *atomic.Value) (*kv.KeyRanges, error) { - keyRanges, err := indexRangesToKVRangesForTablesWithInterruptSignal(sc, []int64{tid}, idxID, ranges, fb, memTracker, interruptSignal) +func IndexRangesToKVRangesWithInterruptSignal(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range, memTracker *memory.Tracker, interruptSignal *atomic.Value) (*kv.KeyRanges, error) { + keyRanges, err := indexRangesToKVRangesForTablesWithInterruptSignal(sc, []int64{tid}, idxID, ranges, memTracker, interruptSignal) if err != nil { return nil, err } @@ -672,52 +642,14 @@ func IndexRangesToKVRangesWithInterruptSignal(sc *stmtctx.StatementContext, tid, } // IndexRangesToKVRangesForTables converts indexes ranges to "KeyRange". -func IndexRangesToKVRangesForTables(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) (*kv.KeyRanges, error) { - return indexRangesToKVRangesForTablesWithInterruptSignal(sc, tids, idxID, ranges, fb, nil, nil) +func IndexRangesToKVRangesForTables(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range) (*kv.KeyRanges, error) { + return indexRangesToKVRangesForTablesWithInterruptSignal(sc, tids, idxID, ranges, nil, nil) } // IndexRangesToKVRangesForTablesWithInterruptSignal converts indexes ranges to "KeyRange". // The process can be interrupted by set `interruptSignal` to true. -func indexRangesToKVRangesForTablesWithInterruptSignal(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback, memTracker *memory.Tracker, interruptSignal *atomic.Value) (*kv.KeyRanges, error) { - if fb == nil || fb.Hist == nil { - return indexRangesToKVWithoutSplit(sc, tids, idxID, ranges, memTracker, interruptSignal) - } - // The following code is non maintained since the feedback deprecated. - feedbackRanges := make([]*ranger.Range, 0, len(ranges)) - for _, ran := range ranges { - low, high, err := EncodeIndexKey(sc, ran) - if err != nil { - return nil, err - } - feedbackRanges = append(feedbackRanges, &ranger.Range{LowVal: []types.Datum{types.NewBytesDatum(low)}, - HighVal: []types.Datum{types.NewBytesDatum(high)}, LowExclude: false, HighExclude: true, Collators: collate.GetBinaryCollatorSlice(1)}) - } - feedbackRanges, ok := fb.Hist.SplitRange(sc, feedbackRanges, true) - if !ok { - fb.Invalidate() - } - krs := make([]kv.KeyRange, 0, len(feedbackRanges)) - for _, ran := range feedbackRanges { - low, high := ran.LowVal[0].GetBytes(), ran.HighVal[0].GetBytes() - if ran.LowExclude { - low = kv.Key(low).PrefixNext() - } - ran.LowVal[0].SetBytes(low) - // If this range is split by histogram, then the high val will equal to one bucket's upper bound, - // since we need to guarantee each range falls inside the exactly one bucket, `PrefixNext` will make the - // high value greater than upper bound, so we store the high value here. - ran.HighVal[0].SetBytes(high) - if !ran.HighExclude { - high = kv.Key(high).PrefixNext() - } - for _, tid := range tids { - startKey := tablecodec.EncodeIndexSeekKey(tid, idxID, low) - endKey := tablecodec.EncodeIndexSeekKey(tid, idxID, high) - krs = append(krs, kv.KeyRange{StartKey: startKey, EndKey: endKey}) - } - } - fb.StoreRanges(feedbackRanges) - return kv.NewNonParitionedKeyRanges(krs), nil +func indexRangesToKVRangesForTablesWithInterruptSignal(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, memTracker *memory.Tracker, interruptSignal *atomic.Value) (*kv.KeyRanges, error) { + return indexRangesToKVWithoutSplit(sc, tids, idxID, ranges, memTracker, interruptSignal) } // CommonHandleRangesToKVRanges converts common handle ranges to "KeyRange". diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index 4db0435da9495..da10f5827156a 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -19,13 +19,10 @@ import ( "github.com/pingcap/tidb/domain/resourcegroup" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/memory" @@ -107,7 +104,7 @@ func TestTableRangesToKVRanges(t *testing.T) { }, } - actual := TableRangesToKVRanges(13, ranges, nil) + actual := TableRangesToKVRanges(13, ranges) expect := []kv.KeyRange{ { StartKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xd, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1}, @@ -192,7 +189,7 @@ func TestIndexRangesToKVRanges(t *testing.T) { }, } - actual, err := IndexRangesToKVRanges(new(stmtctx.StatementContext), 12, 15, ranges, nil) + actual, err := IndexRangesToKVRanges(new(stmtctx.StatementContext), 12, 15, ranges) require.NoError(t, err) for i := range actual.FirstPartitionRange() { require.Equal(t, expect[i], actual.FirstPartitionRange()[i]) @@ -233,7 +230,7 @@ func TestRequestBuilder1(t *testing.T) { }, } - actual, err := (&RequestBuilder{}).SetHandleRanges(nil, 12, false, ranges, nil). + actual, err := (&RequestBuilder{}).SetHandleRanges(nil, 12, false, ranges). SetDAGRequest(&tipb.DAGRequest{}). SetDesc(false). SetKeepOrder(false). @@ -641,8 +638,7 @@ func TestTableRangesToKVRangesWithFbs(t *testing.T) { Collators: collate.GetBinaryCollatorSlice(1), }, } - fb := newTestFb() - actual := TableRangesToKVRanges(0, ranges, fb) + actual := TableRangesToKVRanges(0, ranges) expect := []kv.KeyRange{ { StartKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1}, @@ -663,8 +659,7 @@ func TestIndexRangesToKVRangesWithFbs(t *testing.T) { Collators: collate.GetBinaryCollatorSlice(1), }, } - fb := newTestFb() - actual, err := IndexRangesToKVRanges(new(stmtctx.StatementContext), 0, 0, ranges, fb) + actual, err := IndexRangesToKVRanges(new(stmtctx.StatementContext), 0, 0, ranges) require.NoError(t, err) expect := []kv.KeyRange{ { @@ -726,18 +721,3 @@ func getExpectedRanges(tid int64, hrs []*handleRange) []kv.KeyRange { } return krs } - -func newTestFb() *statistics.QueryFeedback { - hist := statistics.NewHistogram(1, 30, 30, 0, types.NewFieldType(mysql.TypeLonglong), chunk.InitialCapacity, 0) - for i := 0; i < 10; i++ { - hist.Bounds.AppendInt64(0, int64(i)) - hist.Bounds.AppendInt64(0, int64(i+2)) - hist.Buckets = append(hist.Buckets, statistics.Bucket{Repeat: 10, Count: int64(i + 30)}) - } - fb := statistics.NewQueryFeedback(0, hist, 0, false) - lower, upper := types.NewIntDatum(2), types.NewIntDatum(3) - fb.Feedback = []statistics.Feedback{ - {Lower: &lower, Upper: &upper, Count: 1, Repeat: 1}, - } - return fb -} diff --git a/distsql/select_result.go b/distsql/select_result.go index c0ad0f6cdf308..2287cc439e21c 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/copr" "github.com/pingcap/tidb/telemetry" "github.com/pingcap/tidb/types" @@ -300,7 +299,6 @@ type selectResult struct { respChkIdx int respChunkDecoder *chunk.Decoder - feedback *statistics.QueryFeedback partialCount int64 // number of partial results. sqlType string @@ -398,9 +396,7 @@ func (r *selectResult) fetchResp(ctx context.Context) error { for _, warning := range r.selectResp.Warnings { sc.AppendWarning(dbterror.ClassTiKV.Synthesize(terror.ErrCode(warning.Code), warning.Msg)) } - if r.feedback != nil { - r.feedback.Update(resultSubset.GetStartKey(), r.selectResp.OutputCounts, r.selectResp.Ndvs) - } + r.partialCount++ hasStats, ok := resultSubset.(CopRuntimeStats) @@ -451,7 +447,6 @@ func (r *selectResult) NextRaw(ctx context.Context) (data []byte, err error) { resultSubset, err := r.resp.Next(ctx) r.partialCount++ - r.feedback.Invalidate() if resultSubset != nil && err == nil { data = resultSubset.GetData() } @@ -629,9 +624,6 @@ func (r *selectResult) memConsume(bytes int64) { // Close closes selectResult. func (r *selectResult) Close() error { - if r.feedback.Actual() >= 0 { - metrics.DistSQLScanKeysHistogram.Observe(float64(r.feedback.Actual())) - } metrics.DistSQLPartialCountHistogram.Observe(float64(r.partialCount)) respSize := atomic.SwapInt64(&r.selectRespSize, 0) if respSize > 0 { diff --git a/domain/domain.go b/domain/domain.go index 115774c153531..fa161f85aef07 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -2383,16 +2383,12 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) lease := do.statsLease deltaUpdateTicker := time.NewTicker(20 * lease) gcStatsTicker := time.NewTicker(100 * lease) - dumpFeedbackTicker := time.NewTicker(200 * lease) - loadFeedbackTicker := time.NewTicker(5 * lease) loadLockedTablesTicker := time.NewTicker(5 * lease) dumpColStatsUsageTicker := time.NewTicker(100 * lease) readMemTricker := time.NewTicker(memory.ReadMemInterval) statsHandle := do.StatsHandle() defer func() { dumpColStatsUsageTicker.Stop() - loadFeedbackTicker.Stop() - dumpFeedbackTicker.Stop() gcStatsTicker.Stop() deltaUpdateTicker.Stop() readMemTricker.Stop() @@ -2417,26 +2413,11 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) if err != nil { logutil.BgLogger().Debug("dump stats delta failed", zap.Error(err)) } - statsHandle.UpdateErrorRate(do.InfoSchema()) - case <-loadFeedbackTicker.C: - statsHandle.UpdateStatsByLocalFeedback(do.InfoSchema()) - if !owner.IsOwner() { - continue - } - err := statsHandle.HandleUpdateStats(do.InfoSchema()) - if err != nil { - logutil.BgLogger().Debug("update stats using feedback failed", zap.Error(err)) - } case <-loadLockedTablesTicker.C: err := statsHandle.LoadLockedTables() if err != nil { logutil.BgLogger().Debug("load locked table failed", zap.Error(err)) } - case <-dumpFeedbackTicker.C: - err := statsHandle.DumpStatsFeedbackToKV() - if err != nil { - logutil.BgLogger().Debug("dump stats feedback failed", zap.Error(err)) - } case <-gcStatsTicker.C: if !owner.IsOwner() { continue diff --git a/executor/admin.go b/executor/admin.go index 0a5b087c94f21..da0233acc2659 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/terror" plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -135,7 +134,7 @@ func (e *CheckIndexRangeExec) Open(ctx context.Context) error { return err } - e.result, err = distsql.Select(ctx, e.Ctx(), kvReq, e.RetFieldTypes(), statistics.NewQueryFeedback(0, nil, 0, false)) + e.result, err = distsql.Select(ctx, e.Ctx(), kvReq, e.RetFieldTypes()) if err != nil { return err } @@ -290,7 +289,7 @@ func (e *RecoverIndexExec) buildTableScan(ctx context.Context, txn kv.Transactio // Actually, with limitCnt, the match datas maybe only in one region, so let the concurrency to be 1, // avoid unnecessary region scan. kvReq.Concurrency = 1 - result, err := distsql.Select(ctx, e.Ctx(), kvReq, e.columnsTypes(), statistics.NewQueryFeedback(0, nil, 0, false)) + result, err := distsql.Select(ctx, e.Ctx(), kvReq, e.columnsTypes()) if err != nil { return nil, err } @@ -795,7 +794,7 @@ func (e *CleanupIndexExec) buildIndexScan(ctx context.Context, txn kv.Transactio sc := e.Ctx().GetSessionVars().StmtCtx var builder distsql.RequestBuilder ranges := ranger.FullRange() - keyRanges, err := distsql.IndexRangesToKVRanges(sc, e.physicalID, e.index.Meta().ID, ranges, nil) + keyRanges, err := distsql.IndexRangesToKVRanges(sc, e.physicalID, e.index.Meta().ID, ranges) if err != nil { return nil, err } @@ -817,7 +816,7 @@ func (e *CleanupIndexExec) buildIndexScan(ctx context.Context, txn kv.Transactio } kvReq.Concurrency = 1 - result, err := distsql.Select(ctx, e.Ctx(), kvReq, e.getIdxColTypes(), statistics.NewQueryFeedback(0, nil, 0, false)) + result, err := distsql.Select(ctx, e.Ctx(), kvReq, e.getIdxColTypes()) if err != nil { return nil, err } diff --git a/executor/analyze_col.go b/executor/analyze_col.go index c7c7c0e0029e8..a47ebecce5651 100644 --- a/executor/analyze_col.go +++ b/executor/analyze_col.go @@ -108,7 +108,7 @@ func (e *AnalyzeColumnsExec) open(ranges []*ranger.Range) error { func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectResult, error) { var builder distsql.RequestBuilder - reqBuilder := builder.SetHandleRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.TableID.GetStatisticsID()}, e.handleCols != nil && !e.handleCols.IsInt(), ranges, nil) + reqBuilder := builder.SetHandleRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.TableID.GetStatisticsID()}, e.handleCols != nil && !e.handleCols.IsInt(), ranges) builder.SetResourceGroupTagger(e.ctx.GetSessionVars().StmtCtx.GetResourceGroupTagger()) startTS := uint64(math.MaxUint64) isoLevel := kv.RC diff --git a/executor/analyze_idx.go b/executor/analyze_idx.go index bf30d4fd9749d..257127ef09c19 100644 --- a/executor/analyze_idx.go +++ b/executor/analyze_idx.go @@ -139,7 +139,7 @@ func (e *AnalyzeIndexExec) fetchAnalyzeResult(ranges []*ranger.Range, isNullRang var builder distsql.RequestBuilder var kvReqBuilder *distsql.RequestBuilder if e.isCommonHandle && e.idxInfo.Primary { - kvReqBuilder = builder.SetHandleRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.tableID.GetStatisticsID()}, true, ranges, nil) + kvReqBuilder = builder.SetHandleRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.tableID.GetStatisticsID()}, true, ranges) } else { kvReqBuilder = builder.SetIndexRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.tableID.GetStatisticsID()}, e.idxInfo.ID, ranges) } diff --git a/executor/builder.go b/executor/builder.go index f43d8331784b9..d4315002463a5 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3438,17 +3438,6 @@ func (b *executorBuilder) buildIndexNestedLoopHashJoin(v *plannercore.PhysicalIn return idxHash } -// containsLimit tests if the execs contains Limit because we do not know whether `Limit` has consumed all of its' source, -// so the feedback may not be accurate. -func containsLimit(execs []*tipb.Executor) bool { - for _, exec := range execs { - if exec.Limit != nil { - return true - } - } - return false -} - func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableReader) (*TableReaderExecutor, error) { tablePlans := v.TablePlans if v.StoreType == kv.TiFlash { @@ -3499,20 +3488,7 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea batchCop: v.ReadReqType == plannercore.BatchCop, } e.buildVirtualColumnInfo() - if containsLimit(dagReq.Executors) { - e.feedback = statistics.NewQueryFeedback(0, nil, 0, ts.Desc) - } else { - e.feedback = statistics.NewQueryFeedback(getFeedbackStatsTableID(e.Ctx(), tbl), ts.Hist, int64(ts.StatsCount()), ts.Desc) - } - collect := statistics.CollectFeedback(b.ctx.GetSessionVars().StmtCtx, e.feedback, len(ts.Ranges)) - // Do not collect the feedback when the table is the partition table. - if collect && tbl.Meta().Partition != nil { - collect = false - } - if !collect { - e.feedback.Invalidate() - } - e.dagPB.CollectRangeCounts = &collect + if v.StoreType == kv.TiDB && b.ctx.GetSessionVars().User != nil { // User info is used to do privilege check. It is only used in TiDB cluster memory table. e.dagPB.User = &tipb.UserIdentity{ @@ -3845,24 +3821,6 @@ func buildNoRangeIndexReader(b *executorBuilder, v *plannercore.PhysicalIndexRea plans: v.IndexPlans, outputColumns: v.OutputColumns, } - if containsLimit(dagReq.Executors) { - e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) - } else { - tblID := e.physicalTableID - if b.ctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { - tblID = e.table.Meta().ID - } - e.feedback = statistics.NewQueryFeedback(tblID, is.Hist, int64(is.StatsCount()), is.Desc) - } - collect := statistics.CollectFeedback(b.ctx.GetSessionVars().StmtCtx, e.feedback, len(is.Ranges)) - // Do not collect the feedback when the table is the partition table. - if collect && tbl.Meta().Partition != nil { - collect = false - } - if !collect { - e.feedback.Invalidate() - } - e.dagPB.CollectRangeCounts = &collect for _, col := range v.OutputColumns { dagReq.OutputOffsets = append(dagReq.OutputOffsets, uint32(col.Index)) @@ -4052,23 +4010,6 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn avgRowSize: v.GetAvgTableRowSize(), } - if containsLimit(indexReq.Executors) { - e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) - } else { - e.feedback = statistics.NewQueryFeedback(getFeedbackStatsTableID(e.Ctx(), tbl), is.Hist, int64(is.StatsCount()), is.Desc) - } - // Do not collect the feedback for table request. - collectTable := false - e.tableRequest.CollectRangeCounts = &collectTable - collectIndex := statistics.CollectFeedback(b.ctx.GetSessionVars().StmtCtx, e.feedback, len(is.Ranges)) - // Do not collect the feedback when the table is the partition table. - if collectIndex && tbl.Meta().GetPartitionInfo() != nil { - collectIndex = false - } - if !collectIndex { - e.feedback.Invalidate() - } - e.dagPB.CollectRangeCounts = &collectIndex if v.ExtraHandleCol != nil { e.handleIdx = append(e.handleIdx, v.ExtraHandleCol.Index) e.handleCols = []*expression.Column{v.ExtraHandleCol} @@ -4163,7 +4104,6 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd partialDataSizes := make([]float64, 0, partialPlanCount) indexes := make([]*model.IndexInfo, 0, partialPlanCount) descs := make([]bool, 0, partialPlanCount) - feedbacks := make([]*statistics.QueryFeedback, 0, partialPlanCount) ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) isCorColInPartialFilters := make([]bool, 0, partialPlanCount) isCorColInPartialAccess := make([]bool, 0, partialPlanCount) @@ -4171,10 +4111,6 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd var tempReq *tipb.DAGRequest var err error - feedback := statistics.NewQueryFeedback(0, nil, 0, ts.Desc) - feedback.Invalidate() - feedbacks = append(feedbacks, feedback) - if is, ok := v.PartialPlans[i][0].(*plannercore.PhysicalIndexScan); ok { tempReq, err = buildIndexReq(b.ctx, is.Index.Columns, ts.HandleCols.NumCols(), v.PartialPlans[i]) descs = append(descs, is.Desc) @@ -4225,7 +4161,6 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd partialNetDataSizes: partialDataSizes, dataAvgRowSize: v.GetAvgTableRowSize(), dataReaderBuilder: readerBuilder, - feedbacks: feedbacks, paging: paging, handleCols: v.HandleCols, isCorColInPartialFilters: isCorColInPartialFilters, @@ -4545,7 +4480,7 @@ func (h kvRangeBuilderFromRangeAndPartition) buildKeyRangeSeparately(ranges []*r if len(ranges) == 0 { continue } - kvRange, err := distsql.TableHandleRangesToKVRanges(h.sctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && meta.IsCommonHandle, ranges, nil) + kvRange, err := distsql.TableHandleRangesToKVRanges(h.sctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && meta.IsCommonHandle, ranges) if err != nil { return nil, nil, err } @@ -4562,7 +4497,7 @@ func (h kvRangeBuilderFromRangeAndPartition) buildKeyRange(ranges []*ranger.Rang for i, p := range h.partitions { pid := p.GetPhysicalID() meta := p.Meta() - kvRange, err := distsql.TableHandleRangesToKVRanges(h.sctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && meta.IsCommonHandle, ranges, nil) + kvRange, err := distsql.TableHandleRangesToKVRanges(h.sctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && meta.IsCommonHandle, ranges) if err != nil { return nil, err } @@ -4616,7 +4551,7 @@ func (builder *dataReaderBuilder) buildTableReaderBase(ctx context.Context, e *T } e.kvRanges = kvReq.KeyRanges.AppendSelfTo(e.kvRanges) e.resultHandler = &tableResultHandler{} - result, err := builder.SelectResult(ctx, builder.ctx, kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.ID()) + result, err := builder.SelectResult(ctx, builder.ctx, kvReq, retTypes(e), getPhysicalPlanIDs(e.plans), e.ID()) if err != nil { return nil, err } @@ -4900,7 +4835,7 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l if indexID == -1 { tmpKvRanges, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{tableID}, ranges) } else { - tmpKvRanges, err = distsql.IndexRangesToKVRangesWithInterruptSignal(sc, tableID, indexID, ranges, nil, memTracker, interruptSignal) + tmpKvRanges, err = distsql.IndexRangesToKVRangesWithInterruptSignal(sc, tableID, indexID, ranges, memTracker, interruptSignal) } if err != nil { return nil, err @@ -4945,7 +4880,7 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l tmpKeyRanges, err := distsql.CommonHandleRangesToKVRanges(ctx.GetSessionVars().StmtCtx, []int64{tableID}, tmpDatumRanges) return tmpKeyRanges.FirstPartitionRange(), err } - tmpKeyRanges, err := distsql.IndexRangesToKVRangesWithInterruptSignal(ctx.GetSessionVars().StmtCtx, tableID, indexID, tmpDatumRanges, nil, memTracker, interruptSignal) + tmpKeyRanges, err := distsql.IndexRangesToKVRangesWithInterruptSignal(ctx.GetSessionVars().StmtCtx, tableID, indexID, tmpDatumRanges, memTracker, interruptSignal) return tmpKeyRanges.FirstPartitionRange(), err } @@ -5356,13 +5291,6 @@ func getPhysicalTableID(t table.Table) int64 { return t.Meta().ID } -func getFeedbackStatsTableID(ctx sessionctx.Context, t table.Table) int64 { - if p, ok := t.(table.PhysicalTable); ok && !ctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { - return p.GetPhysicalID() - } - return t.Meta().ID -} - func (b *executorBuilder) buildAdminShowTelemetry(v *plannercore.AdminShowTelemetry) exec.Executor { return &AdminShowTelemetryExec{BaseExecutor: exec.NewBaseExecutor(b.ctx, v.Schema(), v.ID())} } diff --git a/executor/checksum.go b/executor/checksum.go index decc77943c8fa..e367ab7862776 100644 --- a/executor/checksum.go +++ b/executor/checksum.go @@ -243,7 +243,7 @@ func (c *checksumContext) buildTableRequest(ctx sessionctx.Context, tableID int6 var builder distsql.RequestBuilder builder.SetResourceGroupTagger(ctx.GetSessionVars().StmtCtx.GetResourceGroupTagger()) - return builder.SetHandleRanges(ctx.GetSessionVars().StmtCtx, tableID, c.TableInfo.IsCommonHandle, ranges, nil). + return builder.SetHandleRanges(ctx.GetSessionVars().StmtCtx, tableID, c.TableInfo.IsCommonHandle, ranges). SetChecksumRequest(checksum). SetStartTS(c.StartTs). SetConcurrency(ctx.GetSessionVars().DistSQLScanConcurrency()). diff --git a/executor/distsql.go b/executor/distsql.go index 06075bb60ed48..71fc5744e889b 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -42,7 +42,6 @@ import ( plannerutil "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -193,8 +192,7 @@ type IndexReaderExecutor struct { // outputColumns are only required by union scan. outputColumns []*expression.Column - feedback *statistics.QueryFeedback - paging bool + paging bool keepOrder bool desc bool @@ -235,7 +233,6 @@ func (e *IndexReaderExecutor) Close() (err error) { if e.dummy { return nil } - e.Ctx().StoreQueryFeedback(e.feedback) return err } @@ -246,11 +243,7 @@ func (e *IndexReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error return nil } - err := e.result.Next(ctx, req) - if err != nil { - e.feedback.Invalidate() - } - return err + return e.result.Next(ctx, req) } // TODO: cleanup this method. @@ -262,7 +255,7 @@ func (e *IndexReaderExecutor) buildKeyRanges(sc *stmtctx.StatementContext, range if e.index.ID == -1 { rRanges, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{physicalID}, ranges) } else { - rRanges, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, ranges, e.feedback) + rRanges, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, ranges) } return rRanges.FirstPartitionRange(), err } @@ -390,12 +383,10 @@ func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) if e.byItems == nil || len(e.partitions) <= 1 { kvReq, err := e.buildKVReq(kvRanges) if err != nil { - e.feedback.Invalidate() return err } - e.result, err = e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.ID()) + e.result, err = e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), getPhysicalPlanIDs(e.plans), e.ID()) if err != nil { - e.feedback.Invalidate() return err } } else { @@ -403,16 +394,14 @@ func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) for _, kvRange := range kvRanges { kvReq, err := e.buildKVReq([]kv.KeyRange{kvRange}) if err != nil { - e.feedback.Invalidate() return err } kvReqs = append(kvReqs, kvReq) } var results []distsql.SelectResult for _, kvReq := range kvReqs { - result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.ID()) + result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), getPhysicalPlanIDs(e.plans), e.ID()) if err != nil { - e.feedback.Invalidate() return err } results = append(results, result) @@ -457,7 +446,6 @@ type IndexLookUpExecutor struct { resultCh chan *lookupTableTask resultCurr *lookupTableTask - feedback *statistics.QueryFeedback // memTracker is used to track the memory usage of this executor. memTracker *memory.Tracker @@ -527,7 +515,6 @@ func (e *IndexLookUpExecutor) Open(ctx context.Context) error { } err = e.buildTableKeyRanges() if err != nil { - e.feedback.Invalidate() return err } @@ -536,17 +523,12 @@ func (e *IndexLookUpExecutor) Open(ctx context.Context) error { return nil } - err = e.open(ctx) - if err != nil { - e.feedback.Invalidate() - } - return err + return e.open(ctx) } func (e *IndexLookUpExecutor) buildTableKeyRanges() (err error) { sc := e.Ctx().GetSessionVars().StmtCtx if e.partitionTableMode { - e.feedback.Invalidate() // feedback for partition tables is not ready e.partitionKVRanges = make([][]kv.KeyRange, 0, len(e.prunedPartitions)) for _, p := range e.prunedPartitions { // TODO: prune and adjust e.ranges for each partition again, since not all e.ranges are suitable for all e.prunedPartitions. @@ -561,7 +543,7 @@ func (e *IndexLookUpExecutor) buildTableKeyRanges() (err error) { if e.index.ID == -1 { kvRange, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{physicalID}, ranges) } else { - kvRange, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, ranges, e.feedback) + kvRange, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, ranges) } if err != nil { return err @@ -574,7 +556,7 @@ func (e *IndexLookUpExecutor) buildTableKeyRanges() (err error) { if e.index.ID == -1 { kvRanges, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{physicalID}, e.ranges) } else { - kvRanges, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, e.ranges, e.feedback) + kvRanges, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, e.ranges) } e.kvRanges = kvRanges.FirstPartitionRange() } @@ -755,7 +737,7 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, workCh chan< worker.syncErr(err) break } - result, err := distsql.SelectWithRuntimeStats(ctx, e.Ctx(), kvReq, tps, e.feedback, getPhysicalPlanIDs(e.idxPlans), idxID) + result, err := distsql.SelectWithRuntimeStats(ctx, e.Ctx(), kvReq, tps, getPhysicalPlanIDs(e.idxPlans), idxID) if err != nil { worker.syncErr(err) break @@ -769,17 +751,14 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, workCh chan< results = []distsql.SelectResult{ssr} } ctx1, cancel := context.WithCancel(ctx) - fetchErr := worker.fetchHandles(ctx1, results) - if fetchErr != nil { // this error is synced in fetchHandles(), don't sync it again - e.feedback.Invalidate() - } + // this error is synced in fetchHandles(), don't sync it again + _ = worker.fetchHandles(ctx1, results) cancel() for _, result := range results { if err := result.Close(); err != nil { logutil.Logger(ctx).Error("close Select result failed", zap.Error(err)) } } - e.Ctx().StoreQueryFeedback(e.feedback) close(workCh) close(e.resultCh) e.idxWorkerWg.Done() @@ -827,7 +806,6 @@ func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, task *lookup readReplicaScope: e.readReplicaScope, isStaleness: e.isStaleness, columns: e.columns, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), corColInFilter: e.corColInTblSide, plans: e.tblPlans, netDataSize: e.avgRowSize * float64(len(task.handles)), diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index faa49266d8628..4111b96db34f5 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -41,7 +41,6 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" plannerutil "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" @@ -122,7 +121,6 @@ type IndexMergeReaderExecutor struct { resultCh chan *indexMergeTableTask resultCurr *indexMergeTableTask - feedbacks []*statistics.QueryFeedback // memTracker is used to track the memory usage of this executor. memTracker *memory.Tracker @@ -183,9 +181,6 @@ func (e *IndexMergeReaderExecutor) Open(_ context.Context) (err error) { return err } } else { - for _, feedback := range e.feedbacks { - feedback.Invalidate() // feedback is not ready for partition tables - } e.partitionKeyRanges = make([][][]kv.KeyRange, len(e.prunedPartitions)) for i, p := range e.prunedPartitions { if e.partitionKeyRanges[i], err = e.buildKeyRangesForTable(p); err != nil { @@ -234,11 +229,11 @@ func (e *IndexMergeReaderExecutor) buildKeyRangesForTable(tbl table.Table) (rang _, ok := plan[0].(*plannercore.PhysicalIndexScan) if !ok { firstPartRanges, secondPartRanges := distsql.SplitRangesAcrossInt64Boundary(e.ranges[i], false, e.descs[i], tbl.Meta().IsCommonHandle) - firstKeyRanges, err := distsql.TableHandleRangesToKVRanges(sc, []int64{getPhysicalTableID(tbl)}, tbl.Meta().IsCommonHandle, firstPartRanges, nil) + firstKeyRanges, err := distsql.TableHandleRangesToKVRanges(sc, []int64{getPhysicalTableID(tbl)}, tbl.Meta().IsCommonHandle, firstPartRanges) if err != nil { return nil, err } - secondKeyRanges, err := distsql.TableHandleRangesToKVRanges(sc, []int64{getPhysicalTableID(tbl)}, tbl.Meta().IsCommonHandle, secondPartRanges, nil) + secondKeyRanges, err := distsql.TableHandleRangesToKVRanges(sc, []int64{getPhysicalTableID(tbl)}, tbl.Meta().IsCommonHandle, secondPartRanges) if err != nil { return nil, err } @@ -246,7 +241,7 @@ func (e *IndexMergeReaderExecutor) buildKeyRangesForTable(tbl table.Table) (rang ranges = append(ranges, keyRanges) continue } - keyRange, err := distsql.IndexRangesToKVRanges(sc, getPhysicalTableID(tbl), e.indexes[i].ID, e.ranges[i], e.feedbacks[i]) + keyRange, err := distsql.IndexRangesToKVRanges(sc, getPhysicalTableID(tbl), e.indexes[i].ID, e.ranges[i]) if err != nil { return nil, err } @@ -418,7 +413,7 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, syncErr(ctx, e.finished, fetchCh, err) return } - result, err := distsql.SelectWithRuntimeStats(ctx, e.Ctx(), kvReq, tps, e.feedbacks[workID], getPhysicalPlanIDs(e.partialPlans[workID]), e.getPartitalPlanID(workID)) + result, err := distsql.SelectWithRuntimeStats(ctx, e.Ctx(), kvReq, tps, getPhysicalPlanIDs(e.partialPlans[workID]), e.getPartitalPlanID(workID)) if err != nil { syncErr(ctx, e.finished, fetchCh, err) return @@ -433,12 +428,9 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, results = []distsql.SelectResult{ssr} } ctx1, cancel := context.WithCancel(ctx) - _, fetchErr := worker.fetchHandles(ctx1, results, exitCh, fetchCh, e.finished, e.handleCols, workID) - if fetchErr != nil { // this error is synced in fetchHandles(), don't sync it again - e.feedbacks[workID].Invalidate() - } + // this error is reported in fetchHandles(), so ignore it here. + _, _ = worker.fetchHandles(ctx1, results, exitCh, fetchCh, e.finished, e.handleCols, workID) cancel() - e.Ctx().StoreQueryFeedback(e.feedbacks[workID]) }, handleWorkerPanic(ctx, e.finished, fetchCh, nil, partialIndexWorkerType), ) @@ -473,7 +465,6 @@ func (e *IndexMergeReaderExecutor) startPartialTableWorker(ctx context.Context, txnScope: e.txnScope, readReplicaScope: e.readReplicaScope, isStaleness: e.isStaleness, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), plans: e.partialPlans[workID], ranges: e.ranges[workID], netDataSize: e.partialNetDataSizes[workID], @@ -547,17 +538,13 @@ func (e *IndexMergeReaderExecutor) startPartialTableWorker(ctx context.Context, // fetch all handles from this table ctx1, cancel := context.WithCancel(ctx) _, fetchErr := worker.fetchHandles(ctx1, exitCh, fetchCh, e.finished, e.handleCols, parTblIdx, workID) - if fetchErr != nil { // this error is synced in fetchHandles, so don't sync it again - e.feedbacks[workID].Invalidate() - } - // release related resources cancel() tableReaderClosed = true if err = worker.tableReader.Close(); err != nil { logutil.Logger(ctx).Error("close Select result failed:", zap.Error(err)) } - e.Ctx().StoreQueryFeedback(e.feedbacks[workID]) + // this error is reported in fetchHandles(), so ignore it here. if fetchErr != nil { break } @@ -791,7 +778,6 @@ func (e *IndexMergeReaderExecutor) buildFinalTableReader(ctx context.Context, tb readReplicaScope: e.readReplicaScope, isStaleness: e.isStaleness, columns: e.columns, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), plans: e.tblPlans, netDataSize: e.dataAvgRowSize * float64(len(handles)), } @@ -916,7 +902,6 @@ func (e *IndexMergeReaderExecutor) Close() error { e.processWorkerWg.Wait() e.finished = nil e.workerStarted = false - // TODO: how to store e.feedbacks return nil } diff --git a/executor/inspection_summary.go b/executor/inspection_summary.go index 30fc542a9898b..d105f890b76a0 100644 --- a/executor/inspection_summary.go +++ b/executor/inspection_summary.go @@ -272,12 +272,9 @@ var inspectionSummaryRules = map[string][]string{ "stats": { "tidb_statistics_auto_analyze_duration", "tidb_statistics_auto_analyze_ops", - "tidb_statistics_dump_feedback_ops", "tidb_statistics_fast_analyze_status", "tidb_statistics_pseudo_estimation_ops", - "tidb_statistics_significant_feedback", "tidb_statistics_stats_inaccuracy_rate", - "tidb_statistics_store_query_feedback_qps", "tidb_statistics_update_stats_ops", }, "gc": { diff --git a/executor/table_reader.go b/executor/table_reader.go index a4c015a174754..4130956ff7f85 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -36,7 +36,6 @@ import ( "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -54,15 +53,15 @@ var _ exec.Executor = &TableReaderExecutor{} // selectResultHook is used to hack distsql.SelectWithRuntimeStats safely for testing. type selectResultHook struct { selectResultFunc func(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, - fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []int) (distsql.SelectResult, error) + fieldTypes []*types.FieldType, copPlanIDs []int) (distsql.SelectResult, error) } func (sr selectResultHook) SelectResult(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, - fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []int, rootPlanID int) (distsql.SelectResult, error) { + fieldTypes []*types.FieldType, copPlanIDs []int, rootPlanID int) (distsql.SelectResult, error) { if sr.selectResultFunc == nil { - return distsql.SelectWithRuntimeStats(ctx, sctx, kvReq, fieldTypes, fb, copPlanIDs, rootPlanID) + return distsql.SelectWithRuntimeStats(ctx, sctx, kvReq, fieldTypes, copPlanIDs, rootPlanID) } - return sr.selectResultFunc(ctx, sctx, kvReq, fieldTypes, fb, copPlanIDs) + return sr.selectResultFunc(ctx, sctx, kvReq, fieldTypes, copPlanIDs) } type kvRangeBuilder interface { @@ -101,7 +100,6 @@ type TableReaderExecutor struct { // resultHandler handles the order of the result. Since (MAXInt64, MAXUint64] stores before [0, MaxInt64] physically // for unsigned int. resultHandler *tableResultHandler - feedback *statistics.QueryFeedback plans []plannercore.PhysicalPlan tablePlan plannercore.PhysicalPlan @@ -184,14 +182,6 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { } e.resultHandler = &tableResultHandler{} - if e.feedback != nil && e.feedback.Hist != nil { - // EncodeInt don't need *statement.Context. - var ok bool - e.ranges, ok = e.feedback.Hist.SplitRange(nil, e.ranges, false) - if !ok { - e.feedback.Invalidate() - } - } firstPartRanges, secondPartRanges := distsql.SplitRangesAcrossInt64Boundary(e.ranges, e.keepOrder, e.desc, e.table.Meta() != nil && e.table.Meta().IsCommonHandle) @@ -227,7 +217,6 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { firstResult, err := e.buildResp(ctx, firstPartRanges) if err != nil { - e.feedback.Invalidate() return err } if len(secondPartRanges) == 0 { @@ -237,7 +226,6 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { var secondResult distsql.SelectResult secondResult, err = e.buildResp(ctx, secondPartRanges) if err != nil { - e.feedback.Invalidate() return err } e.resultHandler.open(firstResult, secondResult) @@ -261,7 +249,6 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error return tableName }), e.ranges) if err := e.resultHandler.nextChunk(ctx, req); err != nil { - e.feedback.Invalidate() return err } @@ -283,7 +270,6 @@ func (e *TableReaderExecutor) Close() error { if e.dummy { return nil } - e.Ctx().StoreQueryFeedback(e.feedback) return err } @@ -299,7 +285,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra } var results []distsql.SelectResult for _, kvReq := range kvReqs { - result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.ID()) + result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), getPhysicalPlanIDs(e.plans), e.ID()) if err != nil { return nil, err } @@ -312,7 +298,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra if err != nil { return nil, err } - result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.ID()) + result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), getPhysicalPlanIDs(e.plans), e.ID()) if err != nil { return nil, err } @@ -327,7 +313,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra } var results []distsql.SelectResult for _, kvReq := range kvReqs { - result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.ID()) + result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), getPhysicalPlanIDs(e.plans), e.ID()) if err != nil { return nil, err } @@ -348,7 +334,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra }) e.kvRanges = kvReq.KeyRanges.AppendSelfTo(e.kvRanges) - result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.ID()) + result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), getPhysicalPlanIDs(e.plans), e.ID()) if err != nil { return nil, err } @@ -442,7 +428,7 @@ func (e *TableReaderExecutor) buildKVReq(ctx context.Context, ranges []*ranger.R } reqBuilder = builder.SetPartitionKeyRanges(kvRange) } else { - reqBuilder = builder.SetHandleRanges(e.Ctx().GetSessionVars().StmtCtx, getPhysicalTableID(e.table), e.table.Meta() != nil && e.table.Meta().IsCommonHandle, ranges, e.feedback) + reqBuilder = builder.SetHandleRanges(e.Ctx().GetSessionVars().StmtCtx, getPhysicalTableID(e.table), e.table.Meta() != nil && e.table.Meta().IsCommonHandle, ranges) } if e.table != nil && e.table.Type().IsClusterTable() { copDestination := infoschema.GetClusterTableCopDestination(e.table.Meta().Name.L) diff --git a/executor/table_readers_required_rows_test.go b/executor/table_readers_required_rows_test.go index 2a18ae7a48cab..8a3367fd5a414 100644 --- a/executor/table_readers_required_rows_test.go +++ b/executor/table_readers_required_rows_test.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -114,7 +113,7 @@ func mockDistsqlSelectCtxGet(ctx context.Context) (totalRows int, expectedRowsRe } func mockSelectResult(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, - fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []int) (distsql.SelectResult, error) { + fieldTypes []*types.FieldType, copPlanIDs []int) (distsql.SelectResult, error) { totalRows, expectedRowsRet := mockDistsqlSelectCtxGet(ctx) return &requiredRowsSelectResult{ retTypes: fieldTypes, diff --git a/executor/test/analyzetest/analyze_test.go b/executor/test/analyzetest/analyze_test.go index 379f437bec51c..ff13f753f912c 100644 --- a/executor/test/analyzetest/analyze_test.go +++ b/executor/test/analyzetest/analyze_test.go @@ -746,19 +746,9 @@ func testAnalyzeIncremental(tk *testkit.TestKit, t *testing.T, dom *domain.Domai // Result should not change. tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 2 1 2 2 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0")) - // Test analyze incremental with feedback. - // paging is not compatible with feedback. tk.MustExec("set @@tidb_enable_paging = off") tk.MustExec("insert into t values (3,3)") - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) is := dom.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) require.NoError(t, err) @@ -767,17 +757,15 @@ func testAnalyzeIncremental(tk *testkit.TestKit, t *testing.T, dom *domain.Domai tk.MustQuery("select * from t where a > 1") h := dom.StatsHandle() require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(is)) require.NoError(t, h.Update(is)) require.NoError(t, h.LoadNeededHistograms()) - tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 3 0 2 2147483647 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0")) + tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 2 1 2 2 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0")) tblStats := h.GetTableStats(tblInfo) val, err := codec.EncodeKey(tk.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(3)) require.NoError(t, err) - require.Equal(t, uint64(1), tblStats.Indices[tblInfo.Indices[0].ID].QueryBytes(nil, val)) - require.False(t, statistics.IsAnalyzed(tblStats.Indices[tblInfo.Indices[0].ID].Flag)) - require.False(t, statistics.IsAnalyzed(tblStats.Columns[tblInfo.Columns[0].ID].Flag)) + require.Equal(t, uint64(0), tblStats.Indices[tblInfo.Indices[0].ID].QueryBytes(nil, val)) + require.True(t, statistics.IsAnalyzed(tblStats.Indices[tblInfo.Indices[0].ID].Flag)) + require.True(t, statistics.IsAnalyzed(tblStats.Columns[tblInfo.Columns[0].ID].Flag)) tk.MustExec("analyze incremental table t index") require.NoError(t, h.LoadNeededHistograms()) diff --git a/executor/test/executor/BUILD.bazel b/executor/test/executor/BUILD.bazel index bee916f714c86..7671d43abe2fa 100644 --- a/executor/test/executor/BUILD.bazel +++ b/executor/test/executor/BUILD.bazel @@ -33,7 +33,6 @@ go_test( "//sessionctx/stmtctx", "//sessionctx/variable", "//sessiontxn", - "//statistics", "//store/driver/error", "//store/mockstore", "//table/tables", diff --git a/executor/test/executor/executor_test.go b/executor/test/executor/executor_test.go index ea80d99a36b56..2e86189d50f3a 100644 --- a/executor/test/executor/executor_test.go +++ b/executor/test/executor/executor_test.go @@ -54,7 +54,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/statistics" error2 "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table/tables" @@ -1559,23 +1558,6 @@ func TestPlanReplayerDumpSingle(t *testing.T) { } } -func TestUnsignedFeedback(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - oriProbability := statistics.FeedbackProbability.Load() - statistics.FeedbackProbability.Store(1.0) - defer func() { statistics.FeedbackProbability.Store(oriProbability) }() - tk.MustExec("use test") - tk.MustExec("create table t(a bigint unsigned, b int, primary key(a))") - tk.MustExec("insert into t values (1,1),(2,2)") - tk.MustExec("analyze table t") - tk.MustQuery("select count(distinct b) from t").Check(testkit.Rows("2")) - result := tk.MustQuery("explain analyze select count(distinct b) from t") - require.Equal(t, "table:t", result.Rows()[2][4]) - require.Equal(t, "keep order:false", result.Rows()[2][6]) -} - func TestAlterTableComment(t *testing.T) { store := testkit.CreateMockStore(t) diff --git a/executor/test/issuetest/BUILD.bazel b/executor/test/issuetest/BUILD.bazel index 2680bd5126f9c..32ce75a0136f4 100644 --- a/executor/test/issuetest/BUILD.bazel +++ b/executor/test/issuetest/BUILD.bazel @@ -19,7 +19,6 @@ go_test( "//parser/mysql", "//session", "//sessionctx/variable", - "//statistics", "//testkit", "//util", "//util/memory", diff --git a/executor/test/issuetest/executor_issue_test.go b/executor/test/issuetest/executor_issue_test.go index d72903bd77b76..e7e997d106b79 100644 --- a/executor/test/issuetest/executor_issue_test.go +++ b/executor/test/issuetest/executor_issue_test.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/memory" @@ -1180,23 +1179,6 @@ func Test15492(t *testing.T) { tk.MustQuery("select a + 1 as field1, a as field2 from t order by field1, field2 limit 2").Check(testkit.Rows("2 1", "3 2")) } -func TestIssue23567(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - oriProbability := statistics.FeedbackProbability.Load() - statistics.FeedbackProbability.Store(1.0) - defer func() { statistics.FeedbackProbability.Store(oriProbability) }() - failpoint.Enable("github.com/pingcap/tidb/statistics/feedbackNoNDVCollect", `return("")`) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a bigint unsigned, b int, primary key(a))") - tk.MustExec("insert into t values (1, 1), (2, 2)") - tk.MustExec("analyze table t") - // The SQL should not panic. - tk.MustQuery("select count(distinct b) from t") - failpoint.Disable("github.com/pingcap/tidb/statistics/feedbackNoNDVCollect") -} - func TestIssue33038(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/infoschema/metric_table_def.go b/infoschema/metric_table_def.go index 0bc722ad6ffb8..08ebcb501721c 100644 --- a/infoschema/metric_table_def.go +++ b/infoschema/metric_table_def.go @@ -541,31 +541,6 @@ var MetricTableMap = map[string]MetricTableDef{ PromQL: "sum(increase(tidb_statistics_pseudo_estimation_total{$LABEL_CONDITIONS}[$RANGE_DURATION]))", Labels: []string{"instance"}, }, - "tidb_statistics_dump_feedback_ops": { - Comment: "TiDB dumping statistics back to kv storage times", - PromQL: "sum(rate(tidb_statistics_dump_feedback_total{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type,instance)", - Labels: []string{"instance", "type"}, - }, - "tidb_statistics_dump_feedback_total_count": { - Comment: "The total count of operations that TiDB dumping statistics back to kv storage", - PromQL: "sum(increase(tidb_statistics_dump_feedback_total{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type,instance)", - Labels: []string{"instance", "type"}, - }, - "tidb_statistics_store_query_feedback_qps": { - Comment: "TiDB store quering feedback counts", - PromQL: "sum(rate(tidb_statistics_store_query_feedback_total{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type,instance) ", - Labels: []string{"instance", "type"}, - }, - "tidb_statistics_store_query_feedback_total_count": { - Comment: "The total count of TiDB store quering feedback", - PromQL: "sum(increase(tidb_statistics_store_query_feedback_total{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type,instance) ", - Labels: []string{"instance", "type"}, - }, - "tidb_statistics_significant_feedback": { - Comment: "Counter of query feedback whose actual count is much different than calculated by current statistics", - PromQL: "sum(rate(tidb_statistics_high_error_rate_feedback_total{$LABEL_CONDITIONS}[$RANGE_DURATION]))", - Labels: []string{"instance"}, - }, "tidb_statistics_update_stats_ops": { Comment: "TiDB updating statistics using feed back counts", PromQL: "sum(rate(tidb_statistics_update_stats_total{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type,instance)", diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 847c3e3fbdfb0..dd8cd7bb0e965 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -51,8 +51,8 @@ "editable": true, "gnetId": null, "graphTooltip": 1, - "id": null, - "iteration": 1655990780337, + "id": 32, + "iteration": 1692177797027, "links": [], "panels": [ { @@ -1598,9 +1598,7 @@ "mode": "time", "name": null, "show": true, - "values": [ - "max" - ] + "values": ["max"] }, "yaxes": [ { @@ -8014,14 +8012,6 @@ } }, { - "type": "graph", - "title": "Mpp Coordinator Counter", - "gridPos": { - "x": 12, - "y": 37, - "w": 12, - "h": 8 - }, "aliasColors": {}, "bars": false, "dashLength": 10, @@ -8034,6 +8024,12 @@ }, "fill": 0, "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 37 + }, "hiddenSeries": false, "id": 311, "legend": { @@ -8078,11 +8074,13 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, + "title": "Mpp Coordinator Counter", "tooltip": { "shared": true, "sort": 0, "value_type": "individual" }, + "type": "graph", "xaxis": { "buckets": null, "mode": "time", @@ -8114,14 +8112,6 @@ } }, { - "type": "graph", - "title": "Mpp Coordinator Latency", - "gridPos": { - "x": 0, - "y": 45, - "w": 12, - "h": 8 - }, "aliasColors": {}, "bars": false, "dashLength": 10, @@ -8134,6 +8124,12 @@ }, "fill": 0, "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 45 + }, "hiddenSeries": false, "id": 312, "legend": { @@ -8199,11 +8195,13 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, + "title": "Mpp Coordinator Latency", "tooltip": { "shared": true, "sort": 0, "value_type": "individual" }, + "type": "graph", "xaxis": { "buckets": null, "mode": "time", @@ -11601,7 +11599,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.10", + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -11757,7 +11755,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.10", + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -11873,7 +11871,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.10", + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -13565,7 +13563,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.10", + "pluginVersion": "7.5.11", "pointradius": 2, "points": false, "renderer": "flot", @@ -13648,13 +13646,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB auto analyze time durations within 95 percent histogram buckets", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 8, "x": 0, - "y": 149 + "y": 12 }, + "hiddenSeries": false, "id": 46, "legend": { "avg": false, @@ -13669,7 +13673,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -13735,13 +13743,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB auto analyze query per second", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 8, "x": 8, - "y": 149 + "y": 12 }, + "hiddenSeries": false, "id": 47, "legend": { "avg": false, @@ -13756,7 +13770,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -13822,13 +13840,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB statistics inaccurate rate", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 8, "x": 16, - "y": 149 + "y": 12 }, + "hiddenSeries": false, "id": 70, "legend": { "avg": false, @@ -13843,7 +13867,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -13923,13 +13951,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB optimizer using pseudo estimation counts", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 8, "x": 0, - "y": 156 + "y": 19 }, + "hiddenSeries": false, "id": 71, "legend": { "avg": false, @@ -13944,7 +13978,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -14003,267 +14041,6 @@ "alignLevel": null } }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB dumping statistics back to kv storage times", - "fill": 1, - "gridPos": { - "h": 7, - "w": 8, - "x": 8, - "y": 156 - }, - "id": 92, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_statistics_dump_feedback_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A", - "step": 30 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Dump Feedback OPS", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB store quering feedback counts", - "fill": 1, - "gridPos": { - "h": 7, - "w": 8, - "x": 16, - "y": 156 - }, - "id": 170, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_statistics_store_query_feedback_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type) ", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A", - "step": 30 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Store Query Feedback QPS", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "Counter of query feedback whose actual count is much different than calculated by current statistics", - "fill": 1, - "gridPos": { - "h": 7, - "w": 8, - "x": 0, - "y": 163 - }, - "id": 113, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_statistics_high_error_rate_feedback_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "Significant Feedback", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Significant Feedback", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, { "aliasColors": {}, "bars": false, @@ -14271,13 +14048,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB updating statistics using feed back counts", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 8, "x": 8, - "y": 163 + "y": 19 }, + "hiddenSeries": false, "id": 93, "legend": { "avg": false, @@ -14292,7 +14075,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -14360,14 +14147,20 @@ "description": "TiDB fast analyze statistics ", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 8, "x": 16, - "y": 163 + "y": 19 }, + "hiddenSeries": false, "id": 173, "legend": { "alignAsTable": true, @@ -14386,7 +14179,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -14463,7 +14260,7 @@ "h": 7, "w": 8, "x": 0, - "y": 170 + "y": 26 }, "hiddenSeries": false, "id": 229, @@ -14484,6 +14281,7 @@ "alertThreshold": true }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -14572,7 +14370,7 @@ "h": 7, "w": 8, "x": 8, - "y": 170 + "y": 26 }, "hiddenSeries": false, "id": 230, @@ -14593,6 +14391,7 @@ "alertThreshold": true }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -14681,7 +14480,7 @@ "h": 7, "w": 8, "x": 16, - "y": 170 + "y": 26 }, "hiddenSeries": false, "id": 233, @@ -14782,7 +14581,7 @@ "h": 7, "w": 12, "x": 0, - "y": 177 + "y": 33 }, "hiddenSeries": false, "id": 234, @@ -14894,7 +14693,7 @@ "h": 7, "w": 12, "x": 12, - "y": 177 + "y": 33 }, "hiddenSeries": false, "id": 235, @@ -14997,7 +14796,7 @@ "h": 7, "w": 8, "x": 0, - "y": 184 + "y": 40 }, "hiddenSeries": false, "id": 236, @@ -15116,7 +14915,7 @@ "h": 7, "w": 8, "x": 8, - "y": 184 + "y": 40 }, "hiddenSeries": false, "id": 237, @@ -19377,7 +19176,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.10", + "pluginVersion": "7.5.11", "pointradius": 2, "points": false, "renderer": "flot", @@ -19534,16 +19333,14 @@ "displayMode": "lcd", "orientation": "horizontal", "reduceOptions": { - "calcs": [ - "lastNotNull" - ], + "calcs": ["lastNotNull"], "fields": "", "values": false }, "showUnfilled": true, "text": {} }, - "pluginVersion": "7.5.10", + "pluginVersion": "7.5.11", "targets": [ { "exemplar": true, @@ -19639,7 +19436,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.10", + "pluginVersion": "7.5.11", "pointradius": 2, "points": false, "renderer": "flot", @@ -19725,7 +19522,7 @@ "h": 1, "w": 24, "x": 0, - "y": 19 + "y": 18 }, "id": 291, "panels": [ @@ -19931,7 +19728,7 @@ "h": 1, "w": 24, "x": 0, - "y": 104 + "y": 19 }, "id": 309, "panels": [ @@ -20387,17 +20184,7 @@ "2h", "1d" ], - "time_options": [ - "5m", - "15m", - "1h", - "6h", - "12h", - "24h", - "2d", - "7d", - "30d" - ] + "time_options": ["5m", "15m", "1h", "6h", "12h", "24h", "2d", "7d", "30d"] }, "timezone": "browser", "title": "Test-Cluster-TiDB", diff --git a/metrics/metrics.go b/metrics/metrics.go index 1120978b0efb5..5957b13f358e5 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -143,14 +143,12 @@ func RegisterMetrics() { prometheus.MustRegister(DistSQLQueryHistogram) prometheus.MustRegister(DistSQLScanKeysHistogram) prometheus.MustRegister(DistSQLScanKeysPartialHistogram) - prometheus.MustRegister(DumpFeedbackCounter) prometheus.MustRegister(ExecuteErrorCounter) prometheus.MustRegister(ExecutorCounter) prometheus.MustRegister(GetTokenDurationHistogram) prometheus.MustRegister(NumOfMultiQueryHistogram) prometheus.MustRegister(HandShakeErrorCounter) prometheus.MustRegister(HandleJobHistogram) - prometheus.MustRegister(SignificantFeedbackCounter) prometheus.MustRegister(FastAnalyzeHistogram) prometheus.MustRegister(SyncLoadCounter) prometheus.MustRegister(SyncLoadTimeoutCounter) @@ -190,7 +188,6 @@ func RegisterMetrics() { prometheus.MustRegister(OngoingTxnDurationHistogram) prometheus.MustRegister(MppCoordinatorStats) prometheus.MustRegister(MppCoordinatorLatency) - prometheus.MustRegister(StoreQueryFeedbackCounter) prometheus.MustRegister(TimeJumpBackCounter) prometheus.MustRegister(TransactionDuration) prometheus.MustRegister(StatementDeadlockDetectDuration) @@ -198,7 +195,6 @@ func RegisterMetrics() { prometheus.MustRegister(StatementLockKeysCount) prometheus.MustRegister(ValidateReadTSFromPDCount) prometheus.MustRegister(UpdateSelfVersionHistogram) - prometheus.MustRegister(UpdateStatsCounter) prometheus.MustRegister(WatchOwnerCounter) prometheus.MustRegister(GCActionRegionResultCounter) prometheus.MustRegister(GCConfigGauge) diff --git a/metrics/stats.go b/metrics/stats.go index eb146bd5a5fcc..4258cc5b30b32 100644 --- a/metrics/stats.go +++ b/metrics/stats.go @@ -20,22 +20,18 @@ import ( // Stats metrics. var ( - AutoAnalyzeHistogram prometheus.Histogram - AutoAnalyzeCounter *prometheus.CounterVec - StatsInaccuracyRate prometheus.Histogram - PseudoEstimation *prometheus.CounterVec - DumpFeedbackCounter *prometheus.CounterVec - UpdateStatsCounter *prometheus.CounterVec - StoreQueryFeedbackCounter *prometheus.CounterVec - SignificantFeedbackCounter prometheus.Counter - FastAnalyzeHistogram *prometheus.HistogramVec - SyncLoadCounter prometheus.Counter - SyncLoadTimeoutCounter prometheus.Counter - SyncLoadHistogram prometheus.Histogram - ReadStatsHistogram prometheus.Histogram - StatsCacheCounter *prometheus.CounterVec - StatsCacheGauge *prometheus.GaugeVec - StatsHealthyGauge *prometheus.GaugeVec + AutoAnalyzeHistogram prometheus.Histogram + AutoAnalyzeCounter *prometheus.CounterVec + StatsInaccuracyRate prometheus.Histogram + PseudoEstimation *prometheus.CounterVec + FastAnalyzeHistogram *prometheus.HistogramVec + SyncLoadCounter prometheus.Counter + SyncLoadTimeoutCounter prometheus.Counter + SyncLoadHistogram prometheus.Histogram + ReadStatsHistogram prometheus.Histogram + StatsCacheCounter *prometheus.CounterVec + StatsCacheGauge *prometheus.GaugeVec + StatsHealthyGauge *prometheus.GaugeVec HistoricalStatsCounter *prometheus.CounterVec PlanReplayerTaskCounter *prometheus.CounterVec @@ -78,38 +74,6 @@ func InitStatsMetrics() { Help: "Counter of pseudo estimation caused by outdated stats.", }, []string{LblType}) - DumpFeedbackCounter = NewCounterVec( - prometheus.CounterOpts{ - Namespace: "tidb", - Subsystem: "statistics", - Name: "dump_feedback_total", - Help: "Counter of dumping feedback.", - }, []string{LblType}) - - UpdateStatsCounter = NewCounterVec( - prometheus.CounterOpts{ - Namespace: "tidb", - Subsystem: "statistics", - Name: "update_stats_total", - Help: "Counter of updating stats using feedback.", - }, []string{LblType}) - - StoreQueryFeedbackCounter = NewCounterVec( - prometheus.CounterOpts{ - Namespace: "tidb", - Subsystem: "statistics", - Name: "store_query_feedback_total", - Help: "Counter of storing query feedback.", - }, []string{LblType}) - - SignificantFeedbackCounter = NewCounter( - prometheus.CounterOpts{ - Namespace: "tidb", - Subsystem: "statistics", - Name: "high_error_rate_feedback_total", - Help: "Counter of query feedback whose actual count is much different than calculated by current statistics", - }) - FastAnalyzeHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index 7833bead1485a..d18fb0424f71a 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -272,7 +272,6 @@ go_test( "//types/parser_driver", "//util", "//util/benchdaily", - "//util/chunk", "//util/collate", "//util/dbterror", "//util/hack", diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 57e3871361c35..b4525849f047a 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1477,13 +1477,6 @@ func (ds *DataSource) buildIndexMergeTableScan(tableFilters []expression.Express if err != nil { return nil, nil, false, err } - if ts.Table.PKIsHandle { - if pkColInfo := ts.Table.GetPkColInfo(); pkColInfo != nil { - if ds.statisticTable.Columns[pkColInfo.ID] != nil { - ts.Hist = &ds.statisticTable.Columns[pkColInfo.ID].Histogram - } - } - } ts.SetStats(ds.tableStats.ScaleByExpectCnt(totalRowCount)) usedStats := ds.SCtx().GetSessionVars().StmtCtx.GetUsedStatsInfo(false) if usedStats != nil && usedStats[ts.physicalTableID] != nil { @@ -2208,13 +2201,6 @@ func (s *LogicalTableScan) GetPhysicalScan(schema *expression.Schema, stats *pro }.Init(s.SCtx(), s.SelectBlockOffset()) ts.SetStats(stats) ts.SetSchema(schema.Clone()) - if ts.Table.PKIsHandle { - if pkColInfo := ts.Table.GetPkColInfo(); pkColInfo != nil { - if ds.statisticTable.Columns[pkColInfo.ID] != nil { - ts.Hist = &ds.statisticTable.Columns[pkColInfo.ID].Histogram - } - } - } return ts } @@ -2672,13 +2658,6 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper filterCondition: slices.Clone(path.TableFilters), }.Init(ds.SCtx(), ds.SelectBlockOffset()) ts.SetSchema(ds.schema.Clone()) - if ts.Table.PKIsHandle { - if pkColInfo := ts.Table.GetPkColInfo(); pkColInfo != nil { - if ds.statisticTable.Columns[pkColInfo.ID] != nil { - ts.Hist = &ds.statisticTable.Columns[pkColInfo.ID].Histogram - } - } - } rowCount := path.CountAfterAccess if prop.ExpectedCnt < ds.StatsInfo().RowCount { selectivity := ds.StatsInfo().RowCount / path.CountAfterAccess @@ -2738,10 +2717,6 @@ func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProper constColsByCond: path.ConstCols, prop: prop, }.Init(ds.SCtx(), ds.SelectBlockOffset()) - statsTbl := ds.statisticTable - if statsTbl.Indices[idx.ID] != nil { - is.Hist = &statsTbl.Indices[idx.ID].Histogram - } rowCount := path.CountAfterAccess is.initSchema(append(path.FullIdxCols, ds.commonHandleCols...), !isSingleScan) diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 5ee4acdcc728a..b361f13afe123 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -608,7 +608,7 @@ func (e *mppTaskGenerator) constructMPPBuildTaskReqForPartitionedTable(ts *Physi for i, p := range partitions { pid := p.GetPhysicalID() meta := p.Meta() - kvRanges, err := distsql.TableHandleRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && ts.Table.IsCommonHandle, splitedRanges, nil) + kvRanges, err := distsql.TableHandleRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && ts.Table.IsCommonHandle, splitedRanges) if err != nil { return nil, nil, errors.Trace(err) } @@ -620,7 +620,7 @@ func (e *mppTaskGenerator) constructMPPBuildTaskReqForPartitionedTable(ts *Physi } func (e *mppTaskGenerator) constructMPPBuildTaskForNonPartitionTable(tid int64, isCommonHandle bool, splitedRanges []*ranger.Range) (*kv.MPPBuildTasksRequest, error) { - kvRanges, err := distsql.TableHandleRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, []int64{tid}, isCommonHandle, splitedRanges, nil) + kvRanges, err := distsql.TableHandleRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, []int64{tid}, isCommonHandle, splitedRanges) if err != nil { return nil, errors.Trace(err) } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index eb716eba5d5f2..2e564f83f9a4e 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -671,10 +671,6 @@ type PhysicalIndexScan struct { // will be different. The schema of index scan will decode all columns of index but the TiDB only need some of them. dataSourceSchema *expression.Schema - // Hist is the histogram when the query was issued. - // It is used for query feedback. - Hist *statistics.Histogram - rangeInfo string // The index scan may be on a partition. @@ -734,9 +730,7 @@ func (p *PhysicalIndexScan) Clone() (PhysicalPlan, error) { if p.dataSourceSchema != nil { cloned.dataSourceSchema = p.dataSourceSchema.Clone() } - if p.Hist != nil { - cloned.Hist = p.Hist.Copy() - } + return cloned, nil } @@ -846,10 +840,6 @@ type PhysicalTableScan struct { TableAsName *model.CIStr - // Hist is the histogram when the query was issued. - // It is used for query feedback. - Hist *statistics.Histogram - physicalTableID int64 rangeInfo string @@ -915,9 +905,6 @@ func (ts *PhysicalTableScan) Clone() (PhysicalPlan, error) { clonedScan.Columns = util.CloneColInfos(ts.Columns) clonedScan.Ranges = util.CloneRanges(ts.Ranges) clonedScan.TableAsName = ts.TableAsName - if ts.Hist != nil { - clonedScan.Hist = ts.Hist.Copy() - } clonedScan.rangeInfo = ts.rangeInfo clonedScan.runtimeFilterList = make([]*RuntimeFilter, len(ts.runtimeFilterList)) for i, rf := range ts.runtimeFilterList { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 366f2dded0f93..ef58e47b40147 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2790,18 +2790,6 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A return nil, err } var commonHandleInfo *model.IndexInfo - // If we want to analyze this table with analyze version 2 but the existing stats is version 1 and stats feedback is enabled, - // we will switch back to analyze version 1. - if statistics.FeedbackProbability.Load() > 0 && version == statistics.Version2 { - statsHandle := domain.GetDomain(b.ctx).StatsHandle() - versionIsSame := statsHandle.CheckAnalyzeVersion(tbl.TableInfo, physicalIDs, &version) - if !versionIsSame { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("Use analyze version 1 on table `%s` "+ - "because this table already has version 1 statistics and query feedback is also enabled. "+ - "If you want to switch to version 2 statistics, please first disable query feedback by setting feedback-probability to 0.0 in the config file.", tbl.Name)) - } - } - if version == statistics.Version2 { p.ColTasks, err = b.buildAnalyzeFullSamplingTask(as, p.ColTasks, physicalIDs, partitionNames, tbl, version, usePersistedOptions, p.OptionsMap) if err != nil { diff --git a/planner/core/planbuilder_test.go b/planner/core/planbuilder_test.go index 6a61bcdd2d84d..5b95414a05943 100644 --- a/planner/core/planbuilder_test.go +++ b/planner/core/planbuilder_test.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" @@ -231,13 +230,11 @@ func TestTablePlansAndTablePlanInPhysicalTableReaderClone(t *testing.T) { col, cst := &expression.Column{RetType: types.NewFieldType(mysql.TypeString)}, &expression.Constant{RetType: types.NewFieldType(mysql.TypeLonglong)} schema := expression.NewSchema(col) tblInfo := &model.TableInfo{} - hist := &statistics.Histogram{Bounds: chunk.New(nil, 0, 0)} // table scan tableScan := &PhysicalTableScan{ AccessCondition: []expression.Expression{col, cst}, Table: tblInfo, - Hist: hist, } tableScan = tableScan.Init(ctx, 0) tableScan.SetSchema(schema) @@ -263,7 +260,6 @@ func TestPhysicalPlanClone(t *testing.T) { schema := expression.NewSchema(col) tblInfo := &model.TableInfo{} idxInfo := &model.IndexInfo{} - hist := &statistics.Histogram{Bounds: chunk.New(nil, 0, 0)} aggDesc1, err := aggregation.NewAggFuncDesc(ctx, ast.AggFuncAvg, []expression.Expression{col}, false) require.NoError(t, err) aggDesc2, err := aggregation.NewAggFuncDesc(ctx, ast.AggFuncCount, []expression.Expression{cst}, true) @@ -274,7 +270,6 @@ func TestPhysicalPlanClone(t *testing.T) { tableScan := &PhysicalTableScan{ AccessCondition: []expression.Expression{col, cst}, Table: tblInfo, - Hist: hist, } tableScan = tableScan.Init(ctx, 0) tableScan.SetSchema(schema) @@ -294,7 +289,6 @@ func TestPhysicalPlanClone(t *testing.T) { AccessCondition: []expression.Expression{col, cst}, Table: tblInfo, Index: idxInfo, - Hist: hist, dataSourceSchema: schema, } indexScan = indexScan.Init(ctx, 0) diff --git a/session/BUILD.bazel b/session/BUILD.bazel index 88c863f13765c..35a5943935d84 100644 --- a/session/BUILD.bazel +++ b/session/BUILD.bazel @@ -59,7 +59,6 @@ go_library( "//sessiontxn", "//sessiontxn/isolation", "//sessiontxn/staleread", - "//statistics", "//statistics/handle", "//store/driver/error", "//store/driver/txn", diff --git a/session/bootstrap.go b/session/bootstrap.go index 62d9cd263ce8b..0730e12e746a1 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -272,6 +272,7 @@ const ( );` // CreateStatsFeedbackTable stores the feedback info which is used to update stats. + // NOTE: Feedback is deprecated, but we still need to create this table for compatibility. CreateStatsFeedbackTable = `CREATE TABLE IF NOT EXISTS mysql.stats_feedback ( table_id BIGINT(64) NOT NULL, is_index TINYINT(2) NOT NULL, @@ -1597,6 +1598,7 @@ func upgradeToVer20(s Session, ver int64) { if ver >= version20 { return } + // NOTE: Feedback is deprecated, but we still need to create this table for compatibility. doReentrantDDL(s, CreateStatsFeedbackTable) } @@ -2844,6 +2846,7 @@ func doDDLWorks(s Session) { // Create gc_delete_range_done table. mustExecute(s, CreateGCDeleteRangeDoneTable) // Create stats_feedback table. + // NOTE: Feedback is deprecated, but we still need to create this table for compatibility. mustExecute(s, CreateStatsFeedbackTable) // Create role_edges table. mustExecute(s, CreateRoleEdgesTable) diff --git a/session/session.go b/session/session.go index 81b864f038fa7..5d74b720b1cd9 100644 --- a/session/session.go +++ b/session/session.go @@ -76,7 +76,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/driver/txn" @@ -446,30 +445,6 @@ func (s *session) GetSessionManager() util.SessionManager { return s.sessionManager } -func (s *session) StoreQueryFeedback(feedback interface{}) { - if variable.FeedbackProbability.Load() <= 0 { - return - } - if fb, ok := feedback.(*statistics.QueryFeedback); !ok || fb == nil || !fb.Valid.Load() { - return - } - if s.statsCollector != nil { - do, err := GetDomain(s.store) - if err != nil { - logutil.BgLogger().Debug("domain not found", zap.Error(err)) - metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblError).Inc() - return - } - err = s.statsCollector.StoreQueryFeedback(feedback, do.StatsHandle(), s.GetSessionVars().GetEnablePseudoForOutdatedStats()) - if err != nil { - logutil.BgLogger().Debug("store query feedback", zap.Error(err)) - metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblError).Inc() - return - } - metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Inc() - } -} - func (s *session) UpdateColStatsUsage(predicateColumns []model.TableItemID) { if s.statsCollector == nil { return diff --git a/sessionctx/context.go b/sessionctx/context.go index 29ba2b0dc29d8..05f1e34bfd18f 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -122,11 +122,7 @@ type Context interface { // GetSessionPlanCache returns the session-level cache of the physical plan. GetSessionPlanCache() PlanCache - // StoreQueryFeedback stores the query feedback. - StoreQueryFeedback(feedback interface{}) - // UpdateColStatsUsage updates the column stats usage. - // TODO: maybe we can use a method called GetSessionStatsCollector to replace both StoreQueryFeedback and UpdateColStatsUsage but we need to deal with import circle if we do so. UpdateColStatsUsage(predicateColumns []model.TableItemID) // HasDirtyContent checks whether there's dirty update on the given table. diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index eeb51efbe9a57..9a31b9eb15181 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -54,7 +54,6 @@ import ( tikvcfg "github.com/tikv/client-go/v2/config" tikvstore "github.com/tikv/client-go/v2/kv" tikvcliutil "github.com/tikv/client-go/v2/util" - atomic2 "go.uber.org/atomic" "go.uber.org/zap" ) @@ -2018,23 +2017,7 @@ var defaultSysVars = []*SysVar{ s.GuaranteeLinearizability = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeVersion, Value: strconv.Itoa(DefTiDBAnalyzeVersion), Type: TypeInt, MinValue: 1, MaxValue: 2, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if normalizedValue == "2" && FeedbackProbability != nil && FeedbackProbability.Load() > 0 { - var original string - var err error - if scope == ScopeGlobal { - original, err = vars.GlobalVarsAccessor.GetGlobalSysVar(TiDBAnalyzeVersion) - if err != nil { - return normalizedValue, nil - } - } else { - original = strconv.Itoa(vars.AnalyzeVersion) - } - vars.StmtCtx.AppendError(errors.New("variable tidb_analyze_version not updated because analyze version 2 is incompatible with query feedback. Please consider setting feedback-probability to 0.0 in config file to disable query feedback")) - return original, nil - } - return normalizedValue, nil - }, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeVersion, Value: strconv.Itoa(DefTiDBAnalyzeVersion), Type: TypeInt, MinValue: 1, MaxValue: 2, SetSession: func(s *SessionVars, val string) error { s.AnalyzeVersion = tidbOptPositiveInt32(val, DefTiDBAnalyzeVersion) return nil }}, @@ -2823,10 +2806,6 @@ func setTiFlashComputeDispatchPolicy(s *SessionVars, val string) error { return nil } -// FeedbackProbability points to the FeedbackProbability in statistics package. -// It's initialized in init() in feedback.go to solve import cycle. -var FeedbackProbability *atomic2.Float64 - // SetNamesVariables is the system variable names related to set names statements. var SetNamesVariables = []string{ CharacterSetClient, diff --git a/statistics/BUILD.bazel b/statistics/BUILD.bazel index 9fdfe41144e49..ff7c8f7cae493 100644 --- a/statistics/BUILD.bazel +++ b/statistics/BUILD.bazel @@ -10,7 +10,6 @@ go_library( "column.go", "debugtrace.go", "estimate.go", - "feedback.go", "fmsketch.go", "histogram.go", "index.go", @@ -28,7 +27,6 @@ go_library( "//config", "//expression", "//kv", - "//metrics", "//parser/ast", "//parser/charset", "//parser/format", @@ -57,7 +55,6 @@ go_library( "//util/tracing", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", - "@com_github_pingcap_log//:log", "@com_github_pingcap_tipb//go-tipb", "@com_github_twmb_murmur3//:murmur3", "@org_golang_x_exp//maps", @@ -73,7 +70,6 @@ go_test( srcs = [ "cmsketch_bench_test.go", "cmsketch_test.go", - "feedback_test.go", "fmsketch_test.go", "histogram_test.go", "integration_test.go", @@ -119,10 +115,8 @@ go_test( "//util/tracing", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", - "@com_github_pingcap_log//:log", "@com_github_stretchr_testify//require", "@com_github_tiancaiamao_gp//:gp", "@org_uber_go_goleak//:goleak", - "@org_uber_go_zap//:zap", ], ) diff --git a/statistics/feedback.go b/statistics/feedback.go deleted file mode 100644 index 585ba3cb4e0ab..0000000000000 --- a/statistics/feedback.go +++ /dev/null @@ -1,1076 +0,0 @@ -// Copyright 2018 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 statistics - -import ( - "bytes" - "encoding/gob" - "math" - "math/rand" - "slices" - "sort" - goatomic "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/mathutil" - "github.com/pingcap/tidb/util/ranger" - "go.uber.org/atomic" - "go.uber.org/zap" -) - -// Feedback represents the total scan count in range [lower, upper). -type Feedback struct { - Lower *types.Datum - Upper *types.Datum - Count int64 - Repeat int64 - Ndv int64 -} - -// QueryFeedback is used to represent the query feedback info. It contains the query's scan ranges and number of rows -// in each range. -type QueryFeedback struct { - Hist *Histogram - Feedback []Feedback - PhysicalID int64 - Tp int - Expected int64 // Expected is the Expected scan count of corresponding query. - actual int64 // actual is the actual scan count of corresponding query. - Valid goatomic.Bool // Valid represents the whether this query feedback is still Valid. - desc bool // desc represents the corresponding query is desc scan. -} - -// NewQueryFeedback returns a new query feedback. -func NewQueryFeedback(physicalID int64, hist *Histogram, expected int64, desc bool) *QueryFeedback { - if hist != nil && hist.Len() == 0 { - hist = nil - } - tp := PkType - if hist != nil && hist.IsIndexHist() { - tp = IndexType - } - rs := &QueryFeedback{ - PhysicalID: physicalID, - Tp: tp, - Hist: hist, - Expected: expected, - desc: desc, - } - rs.Valid.Store(FeedbackProbability.Load() > 0) - return rs -} - -// QueryFeedbackKey is the key for a group of feedbacks on the same index/column. -type QueryFeedbackKey struct { - PhysicalID int64 - HistID int64 - Tp int -} - -// QueryFeedbackMap is the collection of feedbacks. -type QueryFeedbackMap struct { - Feedbacks map[QueryFeedbackKey][]*QueryFeedback - Size int -} - -// NewQueryFeedbackMap builds a feedback collection. -func NewQueryFeedbackMap() *QueryFeedbackMap { - return &QueryFeedbackMap{Feedbacks: make(map[QueryFeedbackKey][]*QueryFeedback)} -} - -// Append adds a feedback into map. -func (m *QueryFeedbackMap) Append(q *QueryFeedback) { - k := QueryFeedbackKey{ - PhysicalID: q.PhysicalID, - HistID: q.Hist.ID, - Tp: q.Tp, - } - m.append(k, []*QueryFeedback{q}) -} - -// MaxQueryFeedbackCount is the max number of feedbacks that are cached in memory. -var MaxQueryFeedbackCount = atomic.NewInt64(1 << 9) - -func (m *QueryFeedbackMap) append(k QueryFeedbackKey, qs []*QueryFeedback) bool { - remained := MaxQueryFeedbackCount.Load() - int64(m.Size) - if remained <= 0 { - return false - } - s, ok := m.Feedbacks[k] - if !ok || s == nil { - s = make([]*QueryFeedback, 0, 8) - } - l := mathutil.Min(int64(len(qs)), remained) - s = append(s, qs[:l]...) - m.Feedbacks[k] = s - m.Size = m.Size + int(l) - return true -} - -// SiftFeedbacks eliminates feedbacks which are overlapped with others. It is a tradeoff between -// feedback accuracy and its overhead. -func (m *QueryFeedbackMap) SiftFeedbacks() { - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - for k, qs := range m.Feedbacks { - fbs := make([]Feedback, 0, len(qs)*2) - for _, q := range qs { - fbs = append(fbs, q.Feedback...) - } - if len(fbs) == 0 { - delete(m.Feedbacks, k) - continue - } - m.Feedbacks[k] = m.Feedbacks[k][:1] - m.Feedbacks[k][0].Feedback, _ = NonOverlappedFeedbacks(sc, fbs) - } - m.Size = len(m.Feedbacks) -} - -// Merge combines 2 collections of feedbacks. -func (m *QueryFeedbackMap) Merge(r *QueryFeedbackMap) { - for k, qs := range r.Feedbacks { - if !m.append(k, qs) { - break - } - } -} - -var ( - // MaxNumberOfRanges is the max number of ranges before split to collect feedback. - MaxNumberOfRanges = 20 - // FeedbackProbability is the probability to collect the feedback. - FeedbackProbability = atomic.NewFloat64(0) -) - -func init() { - // This is for solving import cycle. - // We need to read the value of FeedbackProbability when setting the variable tidb_analyze_version in sessionctx/variable package - // but we have imported sessionctx/variable in statistics package here. - variable.FeedbackProbability = FeedbackProbability -} - -// CalcErrorRate calculates the error rate the current QueryFeedback. -func (q *QueryFeedback) CalcErrorRate() float64 { - expected := float64(q.Expected) - if q.actual == 0 { - if expected == 0 { - return 0 - } - return 1 - } - return math.Abs(expected-float64(q.actual)) / float64(q.actual) -} - -// CollectFeedback decides whether to collect the feedback. It returns false when: -// 1: the feedback is not generated by select query; -// 2: the histogram is nil or has no buckets; -// 3: the number of scan ranges exceeds the limit because it may affect the performance; -// 4: it does not pass the probabilistic sampler. -func CollectFeedback(sc *stmtctx.StatementContext, q *QueryFeedback, numOfRanges int) bool { - if !sc.InSelectStmt { - return false - } - if q.Hist == nil || q.Hist.Len() == 0 { - return false - } - // #nosec G404 - if numOfRanges > MaxNumberOfRanges || rand.Float64() > FeedbackProbability.Load() { - return false - } - return true -} - -// DecodeToRanges decode the feedback to ranges. -func (q *QueryFeedback) DecodeToRanges(isIndex bool) ([]*ranger.Range, error) { - ranges := make([]*ranger.Range, 0, len(q.Feedback)) - for _, val := range q.Feedback { - low, high := *val.Lower, *val.Upper - var lowVal, highVal []types.Datum - if isIndex { - var err error - // As we do not know the origin length, just use a custom value here. - lowVal, _, err = codec.DecodeRange(low.GetBytes(), 4, nil, nil) - if err != nil { - return nil, errors.Trace(err) - } - highVal, _, err = codec.DecodeRange(high.GetBytes(), 4, nil, nil) - if err != nil { - return nil, errors.Trace(err) - } - } else { - _, lowInt, err := codec.DecodeInt(val.Lower.GetBytes()) - if err != nil { - return nil, errors.Trace(err) - } - _, highInt, err := codec.DecodeInt(val.Upper.GetBytes()) - if err != nil { - return nil, errors.Trace(err) - } - lowVal = []types.Datum{types.NewIntDatum(lowInt)} - highVal = []types.Datum{types.NewIntDatum(highInt)} - } - ranges = append(ranges, &(ranger.Range{ - LowVal: lowVal, - HighVal: highVal, - HighExclude: true, - Collators: collate.GetBinaryCollatorSlice(len(lowVal)), - })) - } - return ranges, nil -} - -// DecodeIntValues is called when the current Feedback stores encoded int values. -func (q *QueryFeedback) DecodeIntValues() *QueryFeedback { - nq := &QueryFeedback{} - nq.Feedback = make([]Feedback, 0, len(q.Feedback)) - for _, fb := range q.Feedback { - _, lowInt, err := codec.DecodeInt(fb.Lower.GetBytes()) - if err != nil { - logutil.BgLogger().Debug("decode feedback lower bound value to integer failed", zap.Binary("value", fb.Lower.GetBytes()), zap.Error(err)) - continue - } - _, highInt, err := codec.DecodeInt(fb.Upper.GetBytes()) - if err != nil { - logutil.BgLogger().Debug("decode feedback upper bound value to integer failed", zap.Binary("value", fb.Upper.GetBytes()), zap.Error(err)) - continue - } - low, high := types.NewIntDatum(lowInt), types.NewIntDatum(highInt) - nq.Feedback = append(nq.Feedback, Feedback{Lower: &low, Upper: &high, Count: fb.Count}) - } - return nq -} - -// StoreRanges stores the ranges for update. -func (q *QueryFeedback) StoreRanges(ranges []*ranger.Range) { - q.Feedback = make([]Feedback, 0, len(ranges)) - for _, ran := range ranges { - q.Feedback = append(q.Feedback, Feedback{&ran.LowVal[0], &ran.HighVal[0], 0, 0, 0}) - } -} - -// Invalidate is used to invalidate the query feedback. -func (q *QueryFeedback) Invalidate() { - q.Feedback = nil - q.Hist = nil - q.Valid.Store(false) - q.actual = -1 -} - -// Actual gets the actual row count. -func (q *QueryFeedback) Actual() int64 { - if !q.Valid.Load() { - return -1 - } - return q.actual -} - -// Update updates the query feedback. `startKey` is the start scan key of the partial result, used to find -// the range for update. `counts` is the scan counts of each range, used to update the feedback count info. -func (q *QueryFeedback) Update(startKey kv.Key, counts, ndvs []int64) { - failpoint.Inject("feedbackNoNDVCollect", func() { - ndvs = nil - }) - // Older versions do not have the counts info. - if len(counts) == 0 { - q.Invalidate() - return - } - sum := int64(0) - for _, count := range counts { - sum += count - } - metrics.DistSQLScanKeysPartialHistogram.Observe(float64(sum)) - q.actual += sum - if !q.Valid.Load() || q.Hist == nil { - return - } - - if q.Tp == IndexType { - startKey = tablecodec.CutIndexPrefix(startKey) - } else { - startKey = tablecodec.CutRowKeyPrefix(startKey) - } - // Find the range that startKey falls in. - idx := sort.Search(len(q.Feedback), func(i int) bool { - return bytes.Compare(q.Feedback[i].Lower.GetBytes(), startKey) > 0 - }) - idx-- - if idx < 0 { - return - } - // If the desc is true, the counts is reversed, so here we need to reverse it back. - if q.desc { - for i := 0; i < len(counts)/2; i++ { - j := len(counts) - i - 1 - counts[i], counts[j] = counts[j], counts[i] - } - for i := 0; i < len(ndvs)/2; i++ { - j := len(ndvs) - i - 1 - ndvs[i], ndvs[j] = ndvs[j], ndvs[i] - } - } - // Update the feedback count info. - for i, count := range counts { - if i+idx >= len(q.Feedback) { - q.Invalidate() - break - } - q.Feedback[i+idx].Count += count - } - for i, ndv := range ndvs { - q.Feedback[i+idx].Ndv += ndv - } -} - -// NonOverlappedFeedbacks extracts a set of feedbacks which are not overlapped with each other. -func NonOverlappedFeedbacks(sc *stmtctx.StatementContext, fbs []Feedback) ([]Feedback, bool) { - // Sort feedbacks by end point and start point incrementally, then pick every feedback that is not overlapped - // with the previous chosen feedbacks. - var existsErr bool - slices.SortFunc(fbs, func(i, j Feedback) int { - res, err := i.Upper.Compare(sc, j.Upper, collate.GetBinaryCollator()) - if err != nil { - existsErr = true - } - if existsErr || res != 0 { - return res - } - res, err = i.Lower.Compare(sc, j.Lower, collate.GetBinaryCollator()) - if err != nil { - existsErr = true - } - return res - }) - if existsErr { - return fbs, false - } - resFBs := make([]Feedback, 0, len(fbs)) - previousEnd := &types.Datum{} - for _, fb := range fbs { - res, err := previousEnd.Compare(sc, fb.Lower, collate.GetBinaryCollator()) - if err != nil { - return fbs, false - } - if res <= 0 { - resFBs = append(resFBs, fb) - previousEnd = fb.Upper - } - } - return resFBs, true -} - -// BucketFeedback stands for all the feedback for a bucket. -type BucketFeedback struct { - lower *types.Datum // The lower bound of the new bucket. - upper *types.Datum // The upper bound of the new bucket. - feedback []Feedback // All the feedback info in the same bucket. -} - -// outOfRange checks if the `val` is between `min` and `max`. -func outOfRange(sc *stmtctx.StatementContext, min, max, val *types.Datum) (int, error) { - result, err := val.Compare(sc, min, collate.GetBinaryCollator()) - if err != nil { - return 0, err - } - if result < 0 { - return result, nil - } - result, err = val.Compare(sc, max, collate.GetBinaryCollator()) - if err != nil { - return 0, err - } - if result > 0 { - return result, nil - } - return 0, nil -} - -// adjustFeedbackBoundaries adjust the feedback boundaries according to the `min` and `max`. -// If the feedback has no intersection with `min` and `max`, we could just skip this feedback. -func (f *Feedback) adjustFeedbackBoundaries(sc *stmtctx.StatementContext, min, max *types.Datum) (bool, error) { - result, err := outOfRange(sc, min, max, f.Lower) - if err != nil { - return false, err - } - if result > 0 { - return true, nil - } - if result < 0 { - f.Lower = min - } - result, err = outOfRange(sc, min, max, f.Upper) - if err != nil { - return false, err - } - if result < 0 { - return true, nil - } - if result > 0 { - f.Upper = max - } - return false, nil -} - -// buildBucketFeedback build the feedback for each bucket from the histogram feedback. -func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*BucketFeedback, int) { - bktID2FB := make(map[int]*BucketFeedback) - if len(feedback.Feedback) == 0 { - return bktID2FB, 0 - } - total := 0 - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - min, max := types.GetMinValue(h.Tp), types.GetMaxValue(h.Tp) - for _, fb := range feedback.Feedback { - skip, err := fb.adjustFeedbackBoundaries(sc, &min, &max) - if err != nil { - logutil.BgLogger().Debug("adjust feedback boundaries failed", zap.Error(err)) - continue - } - if skip { - continue - } - idx := h.Bounds.UpperBound(0, fb.Lower) - bktIdx := 0 - // The last bucket also stores the feedback that falls outside the upper bound. - if idx >= h.Bounds.NumRows()-1 { - bktIdx = h.Len() - 1 - } else if h.Len() == 1 { - bktIdx = 0 - } else { - if idx == 0 { - bktIdx = 0 - } else { - bktIdx = (idx - 1) / 2 - } - // Make sure that this feedback lies within the bucket. - if chunk.Compare(h.Bounds.GetRow(2*(bktIdx+1)), 0, fb.Upper) < 0 { - continue - } - } - total++ - bkt := bktID2FB[bktIdx] - if bkt == nil { - bkt = &BucketFeedback{lower: h.GetLower(bktIdx), upper: h.GetUpper(bktIdx)} - bktID2FB[bktIdx] = bkt - } - bkt.feedback = append(bkt.feedback, fb) - // Update the bound if necessary. - res, err := bkt.lower.Compare(nil, fb.Lower, collate.GetBinaryCollator()) - if err != nil { - logutil.BgLogger().Debug("compare datum failed", zap.Any("value1", bkt.lower), zap.Any("value2", fb.Lower), zap.Error(err)) - continue - } - if res > 0 { - bkt.lower = fb.Lower - } - res, err = bkt.upper.Compare(nil, fb.Upper, collate.GetBinaryCollator()) - if err != nil { - logutil.BgLogger().Debug("compare datum failed", zap.Any("value1", bkt.upper), zap.Any("value2", fb.Upper), zap.Error(err)) - continue - } - if res < 0 { - bkt.upper = fb.Upper - } - } - return bktID2FB, total -} - -// getBoundaries gets the new boundaries after split. -func (b *BucketFeedback) getBoundaries(num int) []types.Datum { - // Get all the possible new boundaries. - vals := make([]types.Datum, 0, len(b.feedback)*2+2) - for _, fb := range b.feedback { - vals = append(vals, *fb.Lower, *fb.Upper) - } - vals = append(vals, *b.lower) - err := types.SortDatums(nil, vals) - if err != nil { - logutil.BgLogger().Debug("sort datums failed", zap.Error(err)) - return []types.Datum{*b.lower, *b.upper} - } - total, interval := 0, len(vals)/num - // Pick values per `interval`. - for i := 0; i < len(vals); i, total = i+interval, total+1 { - vals[total] = vals[i] - } - // Append the upper bound. - vals[total] = *b.upper - vals = vals[:total+1] - total = 1 - // Erase the repeat values. - for i := 1; i < len(vals); i++ { - cmp, err := vals[total-1].Compare(nil, &vals[i], collate.GetBinaryCollator()) - if err != nil { - logutil.BgLogger().Debug("compare datum failed", zap.Any("value1", vals[total-1]), zap.Any("value2", vals[i]), zap.Error(err)) - continue - } - if cmp == 0 { - continue - } - vals[total] = vals[i] - total++ - } - return vals[:total] -} - -// There are only two types of datum in bucket: one is `Blob`, which is for index; the other one -// is `Int`, which is for primary key. -type bucket = Feedback - -// splitBucket firstly splits this "BucketFeedback" to "newNumBkts" new buckets, -// calculates the count for each new bucket, merge the new bucket whose count -// is smaller than "minBucketFraction*totalCount" with the next new bucket -// until the last new bucket. -func (b *BucketFeedback) splitBucket(newNumBkts int, totalCount float64, originBucketCount float64, originalNdv int64) []bucket { - // Split the bucket. - bounds := b.getBoundaries(newNumBkts + 1) - bkts := make([]bucket, 0, len(bounds)-1) - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - for i := 1; i < len(bounds); i++ { - newBkt := bucket{&bounds[i-1], bounds[i].Clone(), 0, 0, 0} - // get bucket count - _, ratio := getOverlapFraction(Feedback{b.lower, b.upper, int64(originBucketCount), 0, 0}, newBkt) - countInNewBkt := originBucketCount * ratio - ndvInNewBkt := int64(float64(originalNdv) * ratio) - countInNewBkt, ndvInNewBkt = b.refineBucketCount(sc, newBkt, countInNewBkt, ndvInNewBkt) - // do not split if the count of result bucket is too small. - if countInNewBkt < minBucketFraction*totalCount { - bounds[i] = bounds[i-1] - continue - } - newBkt.Count = int64(countInNewBkt) - newBkt.Ndv = ndvInNewBkt - bkts = append(bkts, newBkt) - // To guarantee that each bucket's range will not overlap. - setNextValue(&bounds[i]) - } - return bkts -} - -// getOverlapFraction gets the overlap fraction of feedback and bucket range. In order to get the bucket count, it also -// returns the ratio between bucket fraction and feedback fraction. -func getOverlapFraction(fb Feedback, bkt bucket) (overlap, ratio float64) { - datums := make([]types.Datum, 0, 4) - datums = append(datums, *fb.Lower, *fb.Upper) - datums = append(datums, *bkt.Lower, *bkt.Upper) - err := types.SortDatums(nil, datums) - if err != nil { - return 0, 0 - } - minValue, maxValue := &datums[0], &datums[3] - fbLower := calcFraction4Datums(minValue, maxValue, fb.Lower) - fbUpper := calcFraction4Datums(minValue, maxValue, fb.Upper) - bktLower := calcFraction4Datums(minValue, maxValue, bkt.Lower) - bktUpper := calcFraction4Datums(minValue, maxValue, bkt.Upper) - ratio = (bktUpper - bktLower) / (fbUpper - fbLower) - // full overlap - if fbLower <= bktLower && bktUpper <= fbUpper { - return bktUpper - bktLower, ratio - } - if bktLower <= fbLower && fbUpper <= bktUpper { - return fbUpper - fbLower, ratio - } - // partial overlap - overlap = math.Min(bktUpper-fbLower, fbUpper-bktLower) - return overlap, ratio -} - -// mergeFullyContainedFeedback merges the max fraction of non-overlapped feedbacks that are fully contained in the bucket. -func (b *BucketFeedback) mergeFullyContainedFeedback(sc *stmtctx.StatementContext, bkt bucket) ( - sumFraction, sumCount float64, ndv int64, ok bool) { - feedbacks := make([]Feedback, 0, len(b.feedback)) - // Get all the fully contained feedbacks. - for _, fb := range b.feedback { - res, err := outOfRange(sc, bkt.Lower, bkt.Upper, fb.Lower) - if res != 0 || err != nil { - return 0, 0, 0, false - } - res, err = outOfRange(sc, bkt.Lower, bkt.Upper, fb.Upper) - if res != 0 || err != nil { - return 0, 0, 0, false - } - feedbacks = append(feedbacks, fb) - } - if len(feedbacks) == 0 { - return 0, 0, 0, false - } - sortedFBs, ok := NonOverlappedFeedbacks(sc, feedbacks) - if !ok { - return 0, 0, 0, false - } - for _, fb := range sortedFBs { - fraction, _ := getOverlapFraction(fb, bkt) - sumFraction += fraction - sumCount += float64(fb.Count) - ndv += fb.Ndv - } - return sumFraction, sumCount, ndv, true -} - -// refineBucketCount refine the newly split bucket count. It uses the feedback that overlaps most -// with the bucket to get the bucket count. -func (b *BucketFeedback) refineBucketCount(sc *stmtctx.StatementContext, bkt bucket, defaultCount float64, defaultNdv int64) (float64, int64) { - bestFraction := minBucketFraction - count := defaultCount - ndv := defaultNdv - sumFraction, sumCount, sumNdv, ok := b.mergeFullyContainedFeedback(sc, bkt) - if ok && sumFraction > bestFraction { - bestFraction = sumFraction - count = sumCount / sumFraction - ndv = int64(float64(sumNdv) / sumFraction) - } - for _, fb := range b.feedback { - fraction, ratio := getOverlapFraction(fb, bkt) - // choose the max overlap fraction - if fraction > bestFraction { - bestFraction = fraction - count = float64(fb.Count) * ratio - ndv = int64(float64(fb.Ndv) * ratio) - } - } - return count, ndv -} - -const ( - defaultSplitCount = 10 - splitPerFeedback = 10 - // defaultBucketCount is the number of buckets a column histogram has. - defaultBucketCount = 256 -) - -// getSplitCount gets the split count for the histogram. It is based on the intuition that: -// 1: If we have more remaining unused buckets, we can split more. -// 2: We cannot split too aggressive, thus we make it split every `splitPerFeedback`. -func getSplitCount(numFeedbacks, remainBuckets int) int { - // Split more if have more buckets available. - splitCount := mathutil.Max(remainBuckets, defaultSplitCount) - return mathutil.Min(splitCount, numFeedbacks/splitPerFeedback) -} - -type bucketScore struct { - id int - score float64 -} - -type bucketScores []bucketScore - -func (bs bucketScores) Len() int { return len(bs) } -func (bs bucketScores) Swap(i, j int) { bs[i], bs[j] = bs[j], bs[i] } -func (bs bucketScores) Less(i, j int) bool { return bs[i].score < bs[j].score } - -const ( - // To avoid the histogram been too imbalanced, we constrain the count of a bucket in range - // [minBucketFraction * totalCount, maxBucketFraction * totalCount]. - minBucketFraction = 1 / 10000.0 - maxBucketFraction = 1 / 10.0 -) - -// getBucketScore gets the score for merge this bucket with previous one. -// TODO: We also need to consider the bucket hit count. -func getBucketScore(bkts []bucket, totalCount float64, id int) bucketScore { - preCount, count := float64(bkts[id-1].Count), float64(bkts[id].Count) - // do not merge if the result bucket is too large - if (preCount + count) > maxBucketFraction*totalCount { - return bucketScore{id, math.MaxFloat64} - } - // Merge them if the result bucket is already too small. - if (preCount + count) < minBucketFraction*totalCount { - return bucketScore{id, 0} - } - low, mid, high := bkts[id-1].Lower, bkts[id-1].Upper, bkts[id].Upper - // If we choose to merge, err is the absolute estimate error for the previous bucket. - err := calcFraction4Datums(low, high, mid)*(preCount+count) - preCount - return bucketScore{id, math.Abs(err / (preCount + count))} -} - -func mergeBuckets(bkts []bucket, isNewBuckets []bool, bucketCount int, totalCount float64) []bucket { - mergeCount := len(bkts) - bucketCount - if mergeCount <= 0 { - return bkts - } - bs := make(bucketScores, 0, len(bkts)) - for i := 1; i < len(bkts); i++ { - // Do not merge the newly created buckets. - if !isNewBuckets[i] && !isNewBuckets[i-1] { - bs = append(bs, getBucketScore(bkts, totalCount, i)) - } - } - sort.Sort(bs) - ids := make([]int, 0, mergeCount) - for i := 0; i < mergeCount; i++ { - ids = append(ids, bs[i].id) - } - slices.Sort(ids) - idCursor, bktCursor := 0, 0 - for i := range bkts { - // Merge this bucket with last one. - if idCursor < mergeCount && ids[idCursor] == i { - bkts[bktCursor-1].Upper = bkts[i].Upper - bkts[bktCursor-1].Count += bkts[i].Count - bkts[bktCursor-1].Repeat = bkts[i].Repeat - bkts[bktCursor-1].Ndv += bkts[i].Ndv - idCursor++ - } else { - bkts[bktCursor] = bkts[i] - bktCursor++ - } - } - bkts = bkts[:bktCursor] - return bkts -} - -// splitBuckets split the histogram buckets according to the feedback. -func splitBuckets(h *Histogram, feedback *QueryFeedback, bucketCount int) ([]bucket, []bool, int64) { - bktID2FB, numTotalFBs := buildBucketFeedback(h, feedback) - buckets := make([]bucket, 0, h.Len()) - isNewBuckets := make([]bool, 0, h.Len()) - splitCount := getSplitCount(numTotalFBs, bucketCount-h.Len()) - for i := 0; i < h.Len(); i++ { - bktFB, ok := bktID2FB[i] - // No feedback, just use the original one. - if !ok { - buckets = append(buckets, bucket{h.GetLower(i), h.GetUpper(i), h.bucketCount(i), h.Buckets[i].Repeat, h.Buckets[i].NDV}) - isNewBuckets = append(isNewBuckets, false) - continue - } - // Distribute the total split count to bucket based on number of bucket feedback. - newBktNums := splitCount * len(bktFB.feedback) / numTotalFBs - bkts := bktFB.splitBucket(newBktNums, h.TotalRowCount(), float64(h.bucketCount(i)), h.Buckets[i].NDV) - buckets = append(buckets, bkts...) - if len(bkts) == 1 { - isNewBuckets = append(isNewBuckets, false) - } else { - for i := 0; i < len(bkts); i++ { - isNewBuckets = append(isNewBuckets, true) - } - } - } - totCount := int64(0) - for _, bkt := range buckets { - totCount += bkt.Count - } - return buckets, isNewBuckets, totCount -} - -// UpdateHistogram updates the histogram according buckets. -func UpdateHistogram(h *Histogram, feedback *QueryFeedback, statsVer int) *Histogram { - return UpdateHistogramWithBucketCount(h, feedback, statsVer, defaultBucketCount) -} - -// UpdateHistogramWithBucketCount updates the histogram according buckets with customized -// bucketCount for testing. -func UpdateHistogramWithBucketCount(h *Histogram, feedback *QueryFeedback, statsVer int, bucketCount int) *Histogram { - if statsVer < Version2 { - // If it's the stats we haven't maintained the bucket NDV yet. Reset the ndv. - for i := range feedback.Feedback { - feedback.Feedback[i].Ndv = 0 - } - } - buckets, isNewBuckets, totalCount := splitBuckets(h, feedback, bucketCount) - buckets = mergeBuckets(buckets, isNewBuckets, bucketCount, float64(totalCount)) - hist := buildNewHistogram(h, buckets) - // Update the NDV of primary key column. - if feedback.Tp == PkType { - hist.NDV = int64(hist.TotalRowCount()) - } else if feedback.Tp == IndexType && statsVer == 2 { - // If we maintained the NDV of bucket. We can also update the total ndv. - totNdv := int64(0) - for _, bkt := range buckets { - totNdv += bkt.Ndv - } - hist.NDV = totNdv - } - return hist -} - -// UpdateCMSketchAndTopN updates the CMSketch and TopN by feedback. -func UpdateCMSketchAndTopN(c *CMSketch, t *TopN, eqFeedbacks []Feedback) (*CMSketch, *TopN) { - if c == nil || len(eqFeedbacks) == 0 { - return c, t - } - newCMSketch := c.Copy() - newTopN := t.Copy() - for _, fb := range eqFeedbacks { - updateValueBytes(newCMSketch, newTopN, fb.Lower.GetBytes(), uint64(fb.Count)) - } - return newCMSketch, newTopN -} - -func buildNewHistogram(h *Histogram, buckets []bucket) *Histogram { - hist := NewHistogram(h.ID, h.NDV, h.NullCount, h.LastUpdateVersion, h.Tp, len(buckets), h.TotColSize) - preCount := int64(0) - for _, bkt := range buckets { - hist.AppendBucketWithNDV(bkt.Lower, bkt.Upper, bkt.Count+preCount, bkt.Repeat, bkt.Ndv) - preCount += bkt.Count - } - return hist -} - -// queryFeedback is used to serialize the QueryFeedback. -type queryFeedback struct { - IntRanges []int64 - // HashValues is the murmur hash values for each index point. - // Note that index points will be stored in `IndexPoints`, we keep it here only for compatibility. - HashValues []uint64 - IndexRanges [][]byte - // IndexPoints stores the value of each equal condition. - IndexPoints [][]byte - // Counts is the number of scan keys in each range. It first stores the count for `IntRanges`, `IndexRanges` or `ColumnRanges`. - // After that, it stores the Ranges for `HashValues`. - Counts []int64 - ColumnRanges [][]byte - - Ndvs []int64 -} - -func encodePKFeedback(q *QueryFeedback) (*queryFeedback, error) { - pb := &queryFeedback{} - for _, fb := range q.Feedback { - // There is no need to update the point queries. - if bytes.Compare(kv.Key(fb.Lower.GetBytes()).PrefixNext(), fb.Upper.GetBytes()) >= 0 { - continue - } - _, low, err := codec.DecodeInt(fb.Lower.GetBytes()) - if err != nil { - return nil, errors.Trace(err) - } - _, high, err := codec.DecodeInt(fb.Upper.GetBytes()) - if err != nil { - return nil, errors.Trace(err) - } - pb.IntRanges = append(pb.IntRanges, low, high) - pb.Counts = append(pb.Counts, fb.Count) - pb.Ndvs = append(pb.Ndvs, fb.Ndv) - } - return pb, nil -} - -func encodeIndexFeedback(q *QueryFeedback) *queryFeedback { - pb := &queryFeedback{} - var pointCounts []int64 - for _, fb := range q.Feedback { - if bytes.Compare(kv.Key(fb.Lower.GetBytes()).PrefixNext(), fb.Upper.GetBytes()) >= 0 { - pb.IndexPoints = append(pb.IndexPoints, fb.Lower.GetBytes()) - pointCounts = append(pointCounts, fb.Count) - pb.Ndvs = append(pb.Ndvs, fb.Ndv) - } else { - pb.IndexRanges = append(pb.IndexRanges, fb.Lower.GetBytes(), fb.Upper.GetBytes()) - pb.Counts = append(pb.Counts, fb.Count) - pb.Ndvs = append(pb.Ndvs, fb.Ndv) - } - } - pb.Counts = append(pb.Counts, pointCounts...) - return pb -} - -func encodeColumnFeedback(q *QueryFeedback) (*queryFeedback, error) { - pb := &queryFeedback{} - sc := stmtctx.StatementContext{TimeZone: time.UTC} - for _, fb := range q.Feedback { - lowerBytes, err := codec.EncodeKey(&sc, nil, *fb.Lower) - if err != nil { - return nil, errors.Trace(err) - } - upperBytes, err := codec.EncodeKey(&sc, nil, *fb.Upper) - if err != nil { - return nil, errors.Trace(err) - } - pb.ColumnRanges = append(pb.ColumnRanges, lowerBytes, upperBytes) - pb.Counts = append(pb.Counts, fb.Count) - } - return pb, nil -} - -// EncodeFeedback encodes the given feedback to byte slice. -func EncodeFeedback(q *QueryFeedback) ([]byte, error) { - var pb *queryFeedback - var err error - switch q.Tp { - case PkType: - pb, err = encodePKFeedback(q) - case IndexType: - pb = encodeIndexFeedback(q) - case ColType: - pb, err = encodeColumnFeedback(q) - } - if err != nil { - return nil, errors.Trace(err) - } - var buf bytes.Buffer - enc := gob.NewEncoder(&buf) - err = enc.Encode(pb) - return buf.Bytes(), errors.Trace(err) -} - -func decodeFeedbackForIndex(q *QueryFeedback, pb *queryFeedback, c *CMSketch, t *TopN) { - q.Tp = IndexType - // decode the index range feedback - for i := 0; i < len(pb.IndexRanges); i += 2 { - lower, upper := types.NewBytesDatum(pb.IndexRanges[i]), types.NewBytesDatum(pb.IndexRanges[i+1]) - q.Feedback = append(q.Feedback, Feedback{&lower, &upper, pb.Counts[i/2], 0, pb.Ndvs[i/2]}) - } - if c != nil { - // decode the index point feedback, just set value count in CM Sketch - start := len(pb.IndexRanges) / 2 - if len(pb.HashValues) > 0 { - for i := 0; i < len(pb.HashValues); i += 2 { - c.setValue(pb.HashValues[i], pb.HashValues[i+1], uint64(pb.Counts[start+i/2])) - } - return - } - for i := 0; i < len(pb.IndexPoints); i++ { - updateValueBytes(c, t, pb.IndexPoints[i], uint64(pb.Counts[start+i])) - } - } -} - -func decodeFeedbackForPK(q *QueryFeedback, pb *queryFeedback, isUnsigned bool) { - q.Tp = PkType - // decode feedback for primary key - for i := 0; i < len(pb.IntRanges); i += 2 { - var lower, upper types.Datum - if isUnsigned { - lower.SetUint64(uint64(pb.IntRanges[i])) - upper.SetUint64(uint64(pb.IntRanges[i+1])) - } else { - lower.SetInt64(pb.IntRanges[i]) - upper.SetInt64(pb.IntRanges[i+1]) - } - q.Feedback = append(q.Feedback, Feedback{&lower, &upper, pb.Counts[i/2], 0, pb.Ndvs[i/2]}) - } -} - -// ConvertDatumsType converts the datums type to `ft`. -func ConvertDatumsType(vals []types.Datum, ft *types.FieldType, loc *time.Location) error { - for i, val := range vals { - if val.Kind() == types.KindMinNotNull || val.Kind() == types.KindMaxValue { - continue - } - newVal, err := tablecodec.UnflattenDatums([]types.Datum{val}, []*types.FieldType{ft}, loc) - if err != nil { - return err - } - vals[i] = newVal[0] - } - return nil -} - -func decodeColumnBounds(data []byte, ft *types.FieldType) ([]types.Datum, error) { - vals, _, err := codec.DecodeRange(data, 1, nil, nil) - if err != nil { - return nil, err - } - err = ConvertDatumsType(vals, ft, time.UTC) - return vals, err -} - -func decodeFeedbackForColumn(q *QueryFeedback, pb *queryFeedback, ft *types.FieldType) error { - q.Tp = ColType - for i := 0; i < len(pb.ColumnRanges); i += 2 { - low, err := decodeColumnBounds(pb.ColumnRanges[i], ft) - if err != nil { - return err - } - high, err := decodeColumnBounds(pb.ColumnRanges[i+1], ft) - if err != nil { - return err - } - q.Feedback = append(q.Feedback, Feedback{&low[0], &high[0], pb.Counts[i/2], 0, 0}) - } - return nil -} - -// DecodeFeedback decodes a byte slice to feedback. -func DecodeFeedback(val []byte, q *QueryFeedback, c *CMSketch, t *TopN, ft *types.FieldType) error { - buf := bytes.NewBuffer(val) - dec := gob.NewDecoder(buf) - pb := &queryFeedback{} - err := dec.Decode(pb) - if err != nil { - return errors.Trace(err) - } - if len(pb.IndexRanges) > 0 || len(pb.HashValues) > 0 || len(pb.IndexPoints) > 0 { - decodeFeedbackForIndex(q, pb, c, t) - } else if len(pb.IntRanges) > 0 { - decodeFeedbackForPK(q, pb, mysql.HasUnsignedFlag(ft.GetFlag())) - } else { - err = decodeFeedbackForColumn(q, pb, ft) - } - return err -} - -// SplitFeedbackByQueryType splits the feedbacks into equality feedbacks and range feedbacks. -func SplitFeedbackByQueryType(feedbacks []Feedback) ([]Feedback, []Feedback) { - var eqFB, ranFB []Feedback - for _, fb := range feedbacks { - // Use `>=` here because sometimes the lower is equal to upper. - if bytes.Compare(kv.Key(fb.Lower.GetBytes()).PrefixNext(), fb.Upper.GetBytes()) >= 0 { - eqFB = append(eqFB, fb) - } else { - ranFB = append(ranFB, fb) - } - } - return eqFB, ranFB -} - -// setNextValue sets the next value for the given datum. For types like float, -// we do not set because it is not discrete and does not matter too much when estimating the scalar info. -func setNextValue(d *types.Datum) { - switch d.Kind() { - case types.KindBytes, types.KindString: - // Here is the encoded value instead of string value, so SetBytes is enough. - d.SetBytes(kv.Key(d.GetBytes()).PrefixNext()) - case types.KindInt64: - d.SetInt64(d.GetInt64() + 1) - case types.KindUint64: - d.SetUint64(d.GetUint64() + 1) - case types.KindMysqlDuration: - duration := d.GetMysqlDuration() - duration.Duration = duration.Duration + 1 - d.SetMysqlDuration(duration) - case types.KindMysqlTime: - t := d.GetMysqlTime() - sc := &stmtctx.StatementContext{TimeZone: types.BoundTimezone} - if _, err := t.Add(sc, types.Duration{Duration: 1, Fsp: 0}); err != nil { - log.Error(errors.ErrorStack(err)) - } - d.SetMysqlTime(t) - } -} - -// SupportColumnType checks if the type of the column can be updated by feedback. -func SupportColumnType(ft *types.FieldType) bool { - switch ft.GetType() { - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeFloat, - mysql.TypeDouble, mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, - mysql.TypeNewDecimal, mysql.TypeDuration, mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: - return true - } - return false -} diff --git a/statistics/feedback_test.go b/statistics/feedback_test.go deleted file mode 100644 index 33366f2805e7e..0000000000000 --- a/statistics/feedback_test.go +++ /dev/null @@ -1,310 +0,0 @@ -// Copyright 2018 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 statistics - -import ( - "bytes" - "testing" - - "github.com/pingcap/log" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/codec" - "github.com/stretchr/testify/require" - "go.uber.org/zap" -) - -func newFeedback(lower, upper, count, ndv int64) Feedback { - low, upp := types.NewIntDatum(lower), types.NewIntDatum(upper) - return Feedback{&low, &upp, count, 0, ndv} -} - -func genFeedbacks(lower, upper int64) []Feedback { - var feedbacks []Feedback - for i := lower; i < upper; i++ { - feedbacks = append(feedbacks, newFeedback(i, upper, upper-i+1, upper-i+1)) - } - return feedbacks -} - -func appendBucket(h *Histogram, l, r int64) { - lower, upper := types.NewIntDatum(l), types.NewIntDatum(r) - h.AppendBucketWithNDV(&lower, &upper, 0, 0, 0) -} - -func genHistogram() *Histogram { - h := NewHistogram(0, 0, 0, 0, types.NewFieldType(mysql.TypeLong), 5, 0) - appendBucket(h, 1, 1) - appendBucket(h, 2, 3) - appendBucket(h, 5, 7) - appendBucket(h, 10, 20) - appendBucket(h, 30, 50) - return h -} - -func TestUpdateHistogram(t *testing.T) { - feedbacks := []Feedback{ - newFeedback(0, 1, 10000, 1), - newFeedback(1, 2, 1, 1), - newFeedback(2, 3, 3, 1), - newFeedback(4, 5, 2, 1), - newFeedback(5, 7, 4, 1), - } - feedbacks = append(feedbacks, genFeedbacks(8, 20)...) - feedbacks = append(feedbacks, genFeedbacks(21, 60)...) - - q := NewQueryFeedback(0, genHistogram(), 0, false) - q.Feedback = feedbacks - require.Equal(t, - "column:0 ndv:10053 totColSize:0\n"+ - "num: 10001 lower_bound: 0 upper_bound: 2 repeats: 0 ndv: 2\n"+ - "num: 7 lower_bound: 2 upper_bound: 5 repeats: 0 ndv: 2\n"+ - "num: 4 lower_bound: 5 upper_bound: 7 repeats: 0 ndv: 1\n"+ - "num: 11 lower_bound: 10 upper_bound: 20 repeats: 0 ndv: 11\n"+ - "num: 19 lower_bound: 30 upper_bound: 49 repeats: 0 ndv: 19\n"+ - "num: 11 lower_bound: 50 upper_bound: 60 repeats: 0 ndv: 11", - UpdateHistogramWithBucketCount(q.Hist, q, Version2, 7).ToString(0)) -} - -func TestSplitBuckets(t *testing.T) { - // test bucket split - feedbacks := []Feedback{newFeedback(0, 1, 1, 1)} - for i := 0; i < 100; i++ { - feedbacks = append(feedbacks, newFeedback(10, 15, 5, 5)) - } - q := NewQueryFeedback(0, genHistogram(), 0, false) - q.Feedback = feedbacks - oldCnts := make([]int64, q.Hist.Len()) - for i := range q.Hist.Buckets { - oldCnts[i] = q.Hist.bucketCount(i) - } - oldNdvs := make([]int64, q.Hist.Len()) - for i := range q.Hist.Buckets { - oldNdvs[i] = q.Hist.Buckets[i].NDV - } - log.Warn("in test", zap.Int64s("ndvs", oldNdvs), zap.Int64s("cnts", oldCnts)) - buckets, isNewBuckets, totalCount := splitBuckets(q.Hist, q, defaultBucketCount) - ndvs := make([]int64, len(buckets)) - for i := range buckets { - ndvs[i] = buckets[i].Ndv - } - log.Warn("in test", zap.Int64s("ndvs", ndvs)) - require.Equal(t, - "column:0 ndv:0 totColSize:0\n"+ - "num: 1 lower_bound: 0 upper_bound: 1 repeats: 0 ndv: 1\n"+ - "num: 0 lower_bound: 2 upper_bound: 3 repeats: 0 ndv: 0\n"+ - "num: 0 lower_bound: 5 upper_bound: 7 repeats: 0 ndv: 0\n"+ - "num: 5 lower_bound: 10 upper_bound: 15 repeats: 0 ndv: 5\n"+ - "num: 0 lower_bound: 16 upper_bound: 20 repeats: 0 ndv: 0\n"+ - "num: 0 lower_bound: 30 upper_bound: 50 repeats: 0 ndv: 0", - buildNewHistogram(q.Hist, buckets).ToString(0)) - require.Equal(t, []bool{false, false, false, true, true, false}, isNewBuckets) - require.Equal(t, int64(6), totalCount) - - // test do not split if the bucket count is too small - feedbacks = []Feedback{newFeedback(0, 1, 100000, 1)} - for i := 0; i < 100; i++ { - feedbacks = append(feedbacks, newFeedback(10, 15, 1, 1)) - } - q = NewQueryFeedback(0, genHistogram(), 0, false) - q.Feedback = feedbacks - buckets, isNewBuckets, totalCount = splitBuckets(q.Hist, q, defaultBucketCount) - require.Equal(t, - "column:0 ndv:0 totColSize:0\n"+ - "num: 100000 lower_bound: 0 upper_bound: 1 repeats: 0 ndv: 1\n"+ - "num: 0 lower_bound: 2 upper_bound: 3 repeats: 0 ndv: 0\n"+ - "num: 0 lower_bound: 5 upper_bound: 7 repeats: 0 ndv: 0\n"+ - "num: 1 lower_bound: 10 upper_bound: 15 repeats: 0 ndv: 1\n"+ - "num: 0 lower_bound: 16 upper_bound: 20 repeats: 0 ndv: 0\n"+ - "num: 0 lower_bound: 30 upper_bound: 50 repeats: 0 ndv: 0", - buildNewHistogram(q.Hist, buckets).ToString(0)) - require.Equal(t, []bool{false, false, false, true, true, false}, isNewBuckets) - require.Equal(t, int64(100001), totalCount) - - // test do not split if the result bucket count is too small - h := NewHistogram(0, 0, 0, 0, types.NewFieldType(mysql.TypeLong), 5, 0) - appendBucket(h, 0, 1000000) - h.Buckets[0].Count = 1000000 - h.Buckets[0].NDV = 1000000 - feedbacks = feedbacks[:0] - for i := 0; i < 100; i++ { - feedbacks = append(feedbacks, newFeedback(0, 10, 1, 1)) - } - q = NewQueryFeedback(0, h, 0, false) - q.Feedback = feedbacks - buckets, isNewBuckets, totalCount = splitBuckets(q.Hist, q, defaultBucketCount) - require.Equal(t, - "column:0 ndv:0 totColSize:0\n"+ - "num: 1000000 lower_bound: 0 upper_bound: 1000000 repeats: 0 ndv: 1000000", - buildNewHistogram(q.Hist, buckets).ToString(0)) - require.Equal(t, []bool{false}, isNewBuckets) - require.Equal(t, int64(1000000), totalCount) - - // test split even if the feedback range is too small - h = NewHistogram(0, 0, 0, 0, types.NewFieldType(mysql.TypeLong), 5, 0) - appendBucket(h, 0, 1000000) - feedbacks = feedbacks[:0] - for i := 0; i < 100; i++ { - feedbacks = append(feedbacks, newFeedback(0, 10, 1, 1)) - } - q = NewQueryFeedback(0, h, 0, false) - q.Feedback = feedbacks - buckets, isNewBuckets, totalCount = splitBuckets(q.Hist, q, defaultBucketCount) - require.Equal(t, - "column:0 ndv:0 totColSize:0\n"+ - "num: 1 lower_bound: 0 upper_bound: 10 repeats: 0 ndv: 1\n"+ - "num: 0 lower_bound: 11 upper_bound: 1000000 repeats: 0 ndv: 0", - buildNewHistogram(q.Hist, buckets).ToString(0)) - require.Equal(t, []bool{true, true}, isNewBuckets) - require.Equal(t, int64(1), totalCount) - - // test merge the non-overlapped feedbacks. - h = NewHistogram(0, 0, 0, 0, types.NewFieldType(mysql.TypeLong), 5, 0) - appendBucket(h, 0, 10000) - feedbacks = feedbacks[:0] - feedbacks = append(feedbacks, newFeedback(0, 4000, 4000, 4000)) - feedbacks = append(feedbacks, newFeedback(4001, 9999, 1000, 1000)) - q = NewQueryFeedback(0, h, 0, false) - q.Feedback = feedbacks - buckets, isNewBuckets, totalCount = splitBuckets(q.Hist, q, defaultBucketCount) - require.Equal(t, - "column:0 ndv:0 totColSize:0\n"+ - "num: 5001 lower_bound: 0 upper_bound: 10000 repeats: 0 ndv: 5001", - buildNewHistogram(q.Hist, buckets).ToString(0)) - require.Equal(t, []bool{false}, isNewBuckets) - require.Equal(t, int64(5001), totalCount) -} - -func TestMergeBuckets(t *testing.T) { - tests := []struct { - points []int64 - counts []int64 - ndvs []int64 - isNewBuckets []bool - bucketCount int - result string - }{ - { - points: []int64{1, 2}, - counts: []int64{1}, - ndvs: []int64{1}, - isNewBuckets: []bool{false}, - bucketCount: 1, - result: "column:0 ndv:0 totColSize:0\nnum: 1 lower_bound: 1 upper_bound: 2 repeats: 0 ndv: 1", - }, - { - points: []int64{1, 2, 2, 3, 3, 4}, - counts: []int64{100000, 1, 1}, - ndvs: []int64{1, 1, 1}, - isNewBuckets: []bool{false, false, false}, - bucketCount: 2, - result: "column:0 ndv:0 totColSize:0\n" + - "num: 100000 lower_bound: 1 upper_bound: 2 repeats: 0 ndv: 1\n" + - "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 2", - }, - // test do not Merge if the result bucket count is too large - { - points: []int64{1, 2, 2, 3, 3, 4, 4, 5}, - counts: []int64{1, 1, 100000, 100000}, - ndvs: []int64{1, 1, 1, 1}, - isNewBuckets: []bool{false, false, false, false}, - bucketCount: 3, - result: "column:0 ndv:0 totColSize:0\n" + - "num: 2 lower_bound: 1 upper_bound: 3 repeats: 0 ndv: 2\n" + - "num: 100000 lower_bound: 3 upper_bound: 4 repeats: 0 ndv: 1\n" + - "num: 100000 lower_bound: 4 upper_bound: 5 repeats: 0 ndv: 1", - }, - } - for _, tt := range tests { - require.Equal(t, len(tt.ndvs), len(tt.counts)) - bkts := make([]bucket, 0, len(tt.counts)) - totalCount := int64(0) - for i := 0; i < len(tt.counts); i++ { - lower, upper := types.NewIntDatum(tt.points[2*i]), types.NewIntDatum(tt.points[2*i+1]) - bkts = append(bkts, bucket{&lower, &upper, tt.counts[i], 0, tt.ndvs[i]}) - totalCount += tt.counts[i] - } - bkts = mergeBuckets(bkts, tt.isNewBuckets, tt.bucketCount, float64(totalCount)) - result := buildNewHistogram(&Histogram{Tp: types.NewFieldType(mysql.TypeLong)}, bkts).ToString(0) - require.Equal(t, tt.result, result) - } -} - -func encodeInt(v int64) *types.Datum { - val := codec.EncodeInt(nil, v) - d := types.NewBytesDatum(val) - return &d -} - -func TestFeedbackEncoding(t *testing.T) { - hist := NewHistogram(0, 0, 0, 0, types.NewFieldType(mysql.TypeLong), 0, 0) - q := &QueryFeedback{Hist: hist, Tp: PkType} - q.Feedback = append(q.Feedback, Feedback{encodeInt(0), encodeInt(3), 1, 0, 1}) - q.Feedback = append(q.Feedback, Feedback{encodeInt(0), encodeInt(5), 1, 0, 1}) - val, err := EncodeFeedback(q) - require.NoError(t, err) - rq := &QueryFeedback{} - require.NoError(t, DecodeFeedback(val, rq, nil, nil, hist.Tp)) - for _, fb := range rq.Feedback { - fb.Lower.SetBytes(codec.EncodeInt(nil, fb.Lower.GetInt64())) - fb.Upper.SetBytes(codec.EncodeInt(nil, fb.Upper.GetInt64())) - } - require.True(t, q.Equal(rq)) - - hist.Tp = types.NewFieldType(mysql.TypeBlob) - q = &QueryFeedback{Hist: hist} - q.Feedback = append(q.Feedback, Feedback{encodeInt(0), encodeInt(3), 1, 0, 1}) - q.Feedback = append(q.Feedback, Feedback{encodeInt(0), encodeInt(1), 1, 0, 1}) - val, err = EncodeFeedback(q) - require.NoError(t, err) - rq = &QueryFeedback{} - cms := NewCMSketch(4, 4) - require.NoError(t, DecodeFeedback(val, rq, cms, nil, hist.Tp)) - require.Equal(t, uint64(1), cms.QueryBytes(codec.EncodeInt(nil, 0))) - q.Feedback = q.Feedback[:1] - require.True(t, q.Equal(rq)) -} - -// Equal tests if two query feedback equal, it is only used in test. -func (q *QueryFeedback) Equal(rq *QueryFeedback) bool { - if len(q.Feedback) != len(rq.Feedback) { - return false - } - for i, fb := range q.Feedback { - rfb := rq.Feedback[i] - if fb.Count != rfb.Count { - return false - } - if fb.Ndv != rfb.Ndv { - return false - } - if fb.Lower.Kind() == types.KindInt64 { - if fb.Lower.GetInt64() != rfb.Lower.GetInt64() { - return false - } - if fb.Upper.GetInt64() != rfb.Upper.GetInt64() { - return false - } - } else { - if !bytes.Equal(fb.Lower.GetBytes(), rfb.Lower.GetBytes()) { - return false - } - if !bytes.Equal(fb.Upper.GetBytes(), rfb.Upper.GetBytes()) { - return false - } - } - } - return true -} diff --git a/statistics/handle/BUILD.bazel b/statistics/handle/BUILD.bazel index 651703064b9e8..a6c84ae37647d 100644 --- a/statistics/handle/BUILD.bazel +++ b/statistics/handle/BUILD.bazel @@ -35,18 +35,14 @@ go_library( "//types", "//util", "//util/chunk", - "//util/codec", - "//util/collate", "//util/logutil", "//util/mathutil", - "//util/ranger", "//util/sqlexec", "//util/syncutil", "//util/timeutil", "@com_github_ngaut_pools//:pools", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", - "@com_github_pingcap_log//:log", "@com_github_pingcap_tipb//go-tipb", "@com_github_tiancaiamao_gp//:gp", "@com_github_tikv_client_go_v2//oracle", diff --git a/statistics/handle/gc.go b/statistics/handle/gc.go index bd233db5b4aaf..b36860478e48c 100644 --- a/statistics/handle/gc.go +++ b/statistics/handle/gc.go @@ -290,9 +290,6 @@ func (h *Handle) DeleteTableStatsFromKV(statsIDs []int64) (err error) { if _, err = exec.ExecuteInternal(ctx, "delete from mysql.stats_top_n where table_id = %?", statsID); err != nil { return err } - if _, err = exec.ExecuteInternal(ctx, "delete from mysql.stats_feedback where table_id = %?", statsID); err != nil { - return err - } if _, err = exec.ExecuteInternal(ctx, "update mysql.stats_extended set version = %?, status = %? where table_id = %? and status in (%?, %?)", startTS, statistics.ExtendedStatsDeleted, statsID, statistics.ExtendedStatsAnalyzed, statistics.ExtendedStatsInited); err != nil { return err } diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index dbc665c72d098..8d594e9e8623e 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -93,12 +93,6 @@ type Handle struct { // written only after acquiring the lock. statsCache *cache.StatsCachePointer - // feedback is used to store query feedback info. - feedback struct { - data *statistics.QueryFeedbackMap - sync.Mutex - } - // globalMap contains all the delta map from collectors when we dump them to KV. globalMap struct { data tableDeltaMap @@ -119,8 +113,6 @@ type Handle struct { mu struct { ctx sessionctx.Context - // rateMap contains the error rate delta from feedback. - rateMap errorRateDeltaMap syncutil.RWMutex } @@ -460,9 +452,6 @@ func (h *Handle) Clear() { for len(h.ddlEventCh) > 0 { <-h.ddlEventCh } - h.feedback.Lock() - h.feedback.data = statistics.NewQueryFeedbackMap() - h.feedback.Unlock() h.mu.ctx.GetSessionVars().InitChunkSize = 1 h.mu.ctx.GetSessionVars().MaxChunkSize = 1 h.mu.ctx.GetSessionVars().EnableChunkRPC = false @@ -474,7 +463,6 @@ func (h *Handle) Clear() { h.colMap.Lock() h.colMap.data = make(colStatsUsageMap) h.colMap.Unlock() - h.mu.rateMap = make(errorRateDeltaMap) h.mu.Unlock() } @@ -499,14 +487,12 @@ func NewHandle(ctx, initStatsCtx sessionctx.Context, lease time.Duration, pool s handle.initStatsCtx = initStatsCtx handle.lease.Store(lease) handle.mu.ctx = ctx - handle.mu.rateMap = make(errorRateDeltaMap) statsCache, err := cache.NewStatsCachePointer() if err != nil { return nil, err } handle.statsCache = statsCache handle.globalMap.data = make(tableDeltaMap) - handle.feedback.data = statistics.NewQueryFeedbackMap() handle.colMap.data = make(colStatsUsageMap) handle.StatsLoad.SubCtxs = make([]sessionctx.Context, cfg.Performance.StatsLoadConcurrency) handle.StatsLoad.NeededItemsCh = make(chan *NeededItemTask, cfg.Performance.StatsLoadQueueSize) @@ -529,16 +515,6 @@ func (h *Handle) SetLease(lease time.Duration) { h.lease.Store(lease) } -// GetQueryFeedback gets the query feedback. It is only used in test. -func (h *Handle) GetQueryFeedback() *statistics.QueryFeedbackMap { - h.feedback.Lock() - defer func() { - h.feedback.data = statistics.NewQueryFeedbackMap() - h.feedback.Unlock() - }() - return h.feedback.data -} - // DurationToTS converts duration to timestamp. func DurationToTS(d time.Duration) uint64 { return oracle.ComposeTS(d.Nanoseconds()/int64(time.Millisecond), 0) @@ -1233,9 +1209,6 @@ func (h *Handle) FlushStats() { if err := h.DumpStatsDeltaToKV(DumpAll); err != nil { logutil.BgLogger().Error("dump stats delta fail", zap.String("category", "stats"), zap.Error(err)) } - if err := h.DumpStatsFeedbackToKV(); err != nil { - logutil.BgLogger().Error("dump stats feedback fail", zap.String("category", "stats"), zap.Error(err)) - } } // TableStatsFromStorage loads table stats info from storage. @@ -1261,16 +1234,6 @@ func (h *Handle) TableStatsFromStorage(tableInfo *model.TableInfo, physicalID in if reader.IsHistory() || statsTbl == nil { return statsTbl, nil } - for histID, idx := range statsTbl.Indices { - if statistics.IsAnalyzed(idx.Flag) { - h.mu.rateMap.clear(physicalID, histID, true) - } - } - for histID, col := range statsTbl.Columns { - if statistics.IsAnalyzed(col.Flag) { - h.mu.rateMap.clear(physicalID, histID, false) - } - } return statsTbl, nil } diff --git a/statistics/handle/handletest/globalstats/BUILD.bazel b/statistics/handle/handletest/globalstats/BUILD.bazel index fa94cf7546096..92ab53fa463ea 100644 --- a/statistics/handle/handletest/globalstats/BUILD.bazel +++ b/statistics/handle/handletest/globalstats/BUILD.bazel @@ -8,12 +8,10 @@ go_test( "main_test.go", ], flaky = True, - shard_count = 11, + shard_count = 10, deps = [ "//parser/model", - "//statistics", "//statistics/handle", - "//statistics/handle/internal", "//testkit", "//testkit/testsetup", "@com_github_stretchr_testify//require", diff --git a/statistics/handle/handletest/globalstats/globalstats_test.go b/statistics/handle/handletest/globalstats/globalstats_test.go index a4f4a2dc22795..3b8e6d95e4f1a 100644 --- a/statistics/handle/handletest/globalstats/globalstats_test.go +++ b/statistics/handle/handletest/globalstats/globalstats_test.go @@ -19,9 +19,7 @@ import ( "testing" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" - "github.com/pingcap/tidb/statistics/handle/internal" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" ) @@ -855,100 +853,6 @@ func TestDDLPartition4GlobalStats(t *testing.T) { require.Equal(t, int64(7), globalStats.RealtimeCount) } -func TestFeedbackWithGlobalStats(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("set @@tidb_analyze_version = 1") - - oriProbability := statistics.FeedbackProbability.Load() - oriNumber := statistics.MaxNumberOfRanges - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - statistics.MaxNumberOfRanges = oriNumber - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - // Case 1: You can't set tidb_analyze_version to 2 if feedback is enabled. - // Note: if we want to set @@tidb_partition_prune_mode = 'dynamic'. We must set tidb_analyze_version to 2 first. We have already tested this. - statistics.FeedbackProbability.Store(1) - testKit.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("1")) - testKit.MustExec("set @@tidb_analyze_version = 2") - testKit.MustQuery("show warnings").Check(testkit.Rows(`Error 1105 variable tidb_analyze_version not updated because analyze version 2 is incompatible with query feedback. Please consider setting feedback-probability to 0.0 in config file to disable query feedback`)) - testKit.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("1")) - - h := dom.StatsHandle() - var err error - // checkFeedbackOnPartitionTable is used to check whether the statistics are the same as before. - checkFeedbackOnPartitionTable := func(statsBefore *statistics.Table, tblInfo *model.TableInfo) { - h.UpdateStatsByLocalFeedback(dom.InfoSchema()) - err = h.DumpStatsFeedbackToKV() - require.NoError(t, err) - err = h.HandleUpdateStats(dom.InfoSchema()) - require.NoError(t, err) - statsTblAfter := h.GetTableStats(tblInfo) - // assert that statistics not changed - // the feedback can not work for the partition table in both static and dynamic mode - internal.AssertTableEqual(t, statsBefore, statsTblAfter) - } - - // Case 2: Feedback wouldn't be applied on version 2 and global-level statistics. - statistics.FeedbackProbability.Store(0) - testKit.MustExec("set @@tidb_analyze_version = 2") - testKit.MustExec("set @@tidb_partition_prune_mode = 'dynamic';") - testKit.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("2")) - testKit.MustExec("create table t (a bigint(64), b bigint(64), index idx(b)) PARTITION BY HASH(a) PARTITIONS 2;") - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t values (1,2),(2,2),(4,5),(2,3),(3,4)") - } - testKit.MustExec("analyze table t with 0 topn") - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - testKit.MustExec("analyze table t") - err = h.Update(dom.InfoSchema()) - require.NoError(t, err) - statsTblBefore := h.GetTableStats(tblInfo) - statistics.FeedbackProbability.Store(1) - // make the statistics inaccurate. - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t values (3,4), (3,4), (3,4), (3,4), (3,4)") - } - // trigger feedback - testKit.MustExec("select b from t partition(p0) use index(idx) where t.b <= 3;") - testKit.MustExec("select b from t partition(p1) use index(idx) where t.b <= 3;") - testKit.MustExec("select b from t use index(idx) where t.b <= 3 order by b;") - testKit.MustExec("select b from t use index(idx) where t.b <= 3;") - checkFeedbackOnPartitionTable(statsTblBefore, tblInfo) - - // Case 3: Feedback is also not effective on version 1 and partition-level statistics. - testKit.MustExec("set tidb_analyze_version = 1") - testKit.MustExec("set @@tidb_partition_prune_mode = 'static';") - testKit.MustExec("create table t1 (a bigint(64), b bigint(64), index idx(b)) PARTITION BY HASH(a) PARTITIONS 2") - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t1 values (1,2),(2,2),(4,5),(2,3),(3,4)") - } - testKit.MustExec("analyze table t1 with 0 topn") - // make the statistics inaccurate. - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t1 values (3,4), (3,4), (3,4), (3,4), (3,4)") - } - is = dom.InfoSchema() - table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - require.NoError(t, err) - tblInfo = table.Meta() - statsTblBefore = h.GetTableStats(tblInfo) - // trigger feedback - testKit.MustExec("select b from t1 partition(p0) use index(idx) where t1.b <= 3;") - testKit.MustExec("select b from t1 partition(p1) use index(idx) where t1.b <= 3;") - testKit.MustExec("select b from t1 use index(idx) where t1.b <= 3 order by b;") - testKit.MustExec("select b from t1 use index(idx) where t1.b <= 3;") - checkFeedbackOnPartitionTable(statsTblBefore, tblInfo) -} - func TestGlobalStatsNDV(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/statistics/handle/historical_stats_handler.go b/statistics/handle/historical_stats_handler.go index 43dff0e8acb05..7829e048b80d9 100644 --- a/statistics/handle/historical_stats_handler.go +++ b/statistics/handle/historical_stats_handler.go @@ -37,8 +37,6 @@ const ( StatsMetaHistorySourceExtendedStats = "extended stats" // StatsMetaHistorySourceSchemaChange indicates stats history meta source from schema change StatsMetaHistorySourceSchemaChange = "schema change" - // StatsMetaHistorySourceFeedBack indicates stats history meta source from feedback - StatsMetaHistorySourceFeedBack = "feedback" ) func recordHistoricalStatsMeta(sctx sessionctx.Context, tableID int64, version uint64, source string) error { diff --git a/statistics/handle/update.go b/statistics/handle/update.go index dfb2353b67df2..6885e3056e439 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -15,7 +15,6 @@ package handle import ( - "bytes" "cmp" "context" "fmt" @@ -28,27 +27,18 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle/cache" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/timeutil" - "go.uber.org/atomic" "go.uber.org/zap" ) @@ -147,24 +137,21 @@ func (m colStatsUsageMap) merge(other colStatsUsageMap) { } } -func merge(s *SessionStatsCollector, deltaMap tableDeltaMap, rateMap errorRateDeltaMap, feedback *statistics.QueryFeedbackMap, colMap colStatsUsageMap) { +func merge(s *SessionStatsCollector, deltaMap tableDeltaMap, rateMap errorRateDeltaMap, colMap colStatsUsageMap) { deltaMap.merge(s.mapper) s.mapper = make(tableDeltaMap) rateMap.merge(s.rateMap) s.rateMap = make(errorRateDeltaMap) - feedback.Merge(s.feedback) - s.feedback = statistics.NewQueryFeedbackMap() colMap.merge(s.colMap) s.colMap = make(colStatsUsageMap) } // SessionStatsCollector is a list item that holds the delta mapper. If you want to write or read mapper, you must lock it. type SessionStatsCollector struct { - mapper tableDeltaMap - feedback *statistics.QueryFeedbackMap - rateMap errorRateDeltaMap - colMap colStatsUsageMap - next *SessionStatsCollector + mapper tableDeltaMap + rateMap errorRateDeltaMap + colMap colStatsUsageMap + next *SessionStatsCollector sync.Mutex // deleted is set to true when a session is closed. Every time we sweep the list, we will remove the useless collector. @@ -185,60 +172,17 @@ func (s *SessionStatsCollector) Update(id int64, delta int64, count int64, colSi s.mapper.update(id, delta, count, colSize) } -// ClearForTest clears the mapper and feedback for test. +// ClearForTest clears the mapper for test. func (s *SessionStatsCollector) ClearForTest() { s.Lock() defer s.Unlock() s.mapper = make(tableDeltaMap) - s.feedback = statistics.NewQueryFeedbackMap() s.rateMap = make(errorRateDeltaMap) s.colMap = make(colStatsUsageMap) s.next = nil s.deleted = false } -var ( - // MinLogScanCount is the minimum scan count for a feedback to be logged. - MinLogScanCount = atomic.NewInt64(1000) - // MinLogErrorRate is the minimum error rate for a feedback to be logged. - MinLogErrorRate = atomic.NewFloat64(0.5) -) - -// StoreQueryFeedback merges the feedback into stats collector. Deprecated. -func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}, h *Handle, enablePseudoForOutdatedStats bool) error { - q := feedback.(*statistics.QueryFeedback) - if !q.Valid.Load() || q.Hist == nil { - return nil - } - - // if table locked, skip - if h.IsTableLocked(q.PhysicalID) { - return nil - } - - err := h.RecalculateExpectCount(q, enablePseudoForOutdatedStats) - if err != nil { - return errors.Trace(err) - } - rate := q.CalcErrorRate() - minScanCnt := MinLogScanCount.Load() - minErrRate := MinLogErrorRate.Load() - if !(rate >= minErrRate && (q.Actual() >= minScanCnt || q.Expected >= minScanCnt)) { - return nil - } - metrics.SignificantFeedbackCounter.Inc() - metrics.StatsInaccuracyRate.Observe(rate) - if log.GetLevel() == zap.DebugLevel { - h.logDetailedInfo(q) - } - s.Lock() - defer s.Unlock() - isIndex := q.Tp == statistics.IndexType - s.rateMap.update(q.PhysicalID, q.Hist.ID, rate, isIndex) - s.feedback.Append(q) - return nil -} - // UpdateColStatsUsage updates the last time when the column stats are used(needed). func (s *SessionStatsCollector) UpdateColStatsUsage(colMap colStatsUsageMap) { s.Lock() @@ -251,11 +195,10 @@ func (h *Handle) NewSessionStatsCollector() *SessionStatsCollector { h.listHead.Lock() defer h.listHead.Unlock() newCollector := &SessionStatsCollector{ - mapper: make(tableDeltaMap), - rateMap: make(errorRateDeltaMap), - next: h.listHead.next, - feedback: statistics.NewQueryFeedbackMap(), - colMap: make(colStatsUsageMap), + mapper: make(tableDeltaMap), + rateMap: make(errorRateDeltaMap), + next: h.listHead.next, + colMap: make(colStatsUsageMap), } h.listHead.next = newCollector return newCollector @@ -466,14 +409,13 @@ const ( func (h *Handle) sweepList() { deltaMap := make(tableDeltaMap) errorRateMap := make(errorRateDeltaMap) - feedback := statistics.NewQueryFeedbackMap() colMap := make(colStatsUsageMap) prev := h.listHead prev.Lock() for curr := prev.next; curr != nil; curr = curr.next { curr.Lock() - // Merge the session stats into deltaMap, errorRateMap and feedback respectively. - merge(curr, deltaMap, errorRateMap, feedback, colMap) + // Merge the session stats into deltaMap, errorRateMap respectively. + merge(curr, deltaMap, errorRateMap, colMap) if curr.deleted { prev.next = curr.next // Since the session is already closed, we can safely unlock it here. @@ -488,13 +430,6 @@ func (h *Handle) sweepList() { h.globalMap.Lock() h.globalMap.data.merge(deltaMap) h.globalMap.Unlock() - h.mu.Lock() - h.mu.rateMap.merge(errorRateMap) - h.mu.Unlock() - h.feedback.Lock() - h.feedback.data.Merge(feedback) - h.feedback.data.SiftFeedbacks() - h.feedback.Unlock() h.colMap.Lock() h.colMap.data.merge(colMap) h.colMap.Unlock() @@ -641,304 +576,6 @@ func (h *Handle) dumpTableStatColSizeToKV(id int64, delta variable.TableDelta) e return errors.Trace(err) } -// DumpStatsFeedbackToKV dumps the stats feedback to KV. Deprecated. -func (h *Handle) DumpStatsFeedbackToKV() error { - h.feedback.Lock() - feedback := h.feedback.data - h.feedback.data = statistics.NewQueryFeedbackMap() - h.feedback.Unlock() - var err error - for _, fbs := range feedback.Feedbacks { - for _, fb := range fbs { - if fb.Tp == statistics.PkType { - err = h.DumpFeedbackToKV(fb) - } else { - t, ok := h.statsCache.Load().GetFromInternal(fb.PhysicalID) - if !ok { - continue - } - idx, ok := t.Indices[fb.Hist.ID] - if !ok { - continue - } - if idx.GetStatsVer() == statistics.Version1 { - err = h.DumpFeedbackForIndex(fb, t) - } else { - err = h.DumpFeedbackToKV(fb) - } - } - if err != nil { - // For simplicity, we just drop other feedbacks in case of error. - break - } - } - } - return errors.Trace(err) -} - -// DumpFeedbackToKV dumps the given feedback to physical kv layer. -func (h *Handle) DumpFeedbackToKV(fb *statistics.QueryFeedback) error { - vals, err := statistics.EncodeFeedback(fb) - if err != nil { - logutil.BgLogger().Debug("error occurred when encoding feedback", zap.Error(err)) - return nil - } - var isIndex int64 - if fb.Tp == statistics.IndexType { - isIndex = 1 - } - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) - const sql = "insert into mysql.stats_feedback (table_id, hist_id, is_index, feedback) values (%?, %?, %?, %?)" - h.mu.Lock() - _, err = h.mu.ctx.(sqlexec.SQLExecutor).ExecuteInternal(ctx, sql, fb.PhysicalID, fb.Hist.ID, isIndex, vals) - h.mu.Unlock() - if err != nil { - metrics.DumpFeedbackCounter.WithLabelValues(metrics.LblError).Inc() - } else { - metrics.DumpFeedbackCounter.WithLabelValues(metrics.LblOK).Inc() - } - return errors.Trace(err) -} - -// UpdateStatsByLocalFeedback will update statistics by the local feedback. -// Currently, we dump the feedback with the period of 10 minutes, which means -// it takes 10 minutes for a feedback to take effect. However, we can use the -// feedback locally on this tidb-server, so it could be used more timely. -func (h *Handle) UpdateStatsByLocalFeedback(is infoschema.InfoSchema) { - h.sweepList() - h.feedback.Lock() - feedback := h.feedback.data - h.feedback.data = statistics.NewQueryFeedbackMap() - h.feedback.Unlock() -OUTER: - for _, fbs := range feedback.Feedbacks { - for _, fb := range fbs { - table, ok := h.getTableByPhysicalID(is, fb.PhysicalID) - if !ok { - continue - } - if table.Meta().Partition != nil { - // If the table is partition table, the feedback will not work. - continue - } - tblStats := h.GetPartitionStats(table.Meta(), fb.PhysicalID) - newTblStats := tblStats.Copy() - if fb.Tp == statistics.IndexType { - idx, ok := tblStats.Indices[fb.Hist.ID] - if !ok || idx.Histogram.Len() == 0 { - continue - } - newIdx := *idx - eqFB, ranFB := statistics.SplitFeedbackByQueryType(fb.Feedback) - if idx.GetStatsVer() >= statistics.Version2 { - // // For StatsVersion higher than Version1, the topn is extracted out of histogram. So we don't update the histogram if the feedback overlaps with some topn. - // ranFB = statistics.CleanRangeFeedbackByTopN(ranFB, idx.TopN) - continue OUTER - } - newIdx.CMSketch, newIdx.TopN = statistics.UpdateCMSketchAndTopN(idx.CMSketch, idx.TopN, eqFB) - newIdx.Histogram = *statistics.UpdateHistogram(&idx.Histogram, &statistics.QueryFeedback{Feedback: ranFB}, int(idx.GetStatsVer())) - newIdx.Histogram.PreCalculateScalar() - newIdx.Flag = statistics.ResetAnalyzeFlag(newIdx.Flag) - newTblStats.Indices[fb.Hist.ID] = &newIdx - } else { - col, ok := tblStats.Columns[fb.Hist.ID] - if !ok || col.Histogram.Len() == 0 { - continue - } - if col.GetStatsVer() >= statistics.Version2 { - // // For StatsVersion higher than Version1, the topn is extracted out of histogram. So we don't update the histogram if the feedback overlaps with some topn. - // ranFB = statistics.CleanRangeFeedbackByTopN(ranFB, idx.TopN) - continue OUTER - } - newCol := *col - // only use the range query to update primary key - _, ranFB := statistics.SplitFeedbackByQueryType(fb.Feedback) - newFB := &statistics.QueryFeedback{Feedback: ranFB} - newFB = newFB.DecodeIntValues() - newCol.Histogram = *statistics.UpdateHistogram(&col.Histogram, newFB, statistics.Version1) - newCol.Flag = statistics.ResetAnalyzeFlag(newCol.Flag) - newTblStats.Columns[fb.Hist.ID] = &newCol - } - for retry := updateStatsCacheRetryCnt; retry > 0; retry-- { - oldCache := h.statsCache.Load() - if h.updateStatsCache(oldCache, []*statistics.Table{newTblStats}, nil) { - break - } - } - } - } -} - -// UpdateErrorRate updates the error rate of columns from h.rateMap to cache. -func (h *Handle) UpdateErrorRate(is infoschema.InfoSchema) { - h.mu.Lock() - tbls := make([]*statistics.Table, 0, len(h.mu.rateMap)) - for id, item := range h.mu.rateMap { - table, ok := h.getTableByPhysicalID(is, id) - if !ok { - continue - } - tbl := h.GetPartitionStats(table.Meta(), id).Copy() - if item.PkErrorRate != nil && tbl.Columns[item.PkID] != nil { - col := *tbl.Columns[item.PkID] - col.ErrorRate.Merge(item.PkErrorRate) - tbl.Columns[item.PkID] = &col - } - for key, val := range item.IdxErrorRate { - if tbl.Indices[key] == nil { - continue - } - idx := *tbl.Indices[key] - idx.ErrorRate.Merge(val) - tbl.Indices[key] = &idx - } - tbls = append(tbls, tbl) - delete(h.mu.rateMap, id) - } - h.mu.Unlock() - for retry := updateStatsCacheRetryCnt; retry > 0; retry-- { - oldCache := h.statsCache.Load() - if h.updateStatsCache(oldCache, tbls, nil) { - break - } - } -} - -// HandleUpdateStats update the stats using feedback. -func (h *Handle) HandleUpdateStats(is infoschema.InfoSchema) error { - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) - tables, _, err := h.execRestrictedSQL(ctx, "SELECT distinct table_id from mysql.stats_feedback") - if err != nil { - return errors.Trace(err) - } - if len(tables) == 0 { - return nil - } - - for _, ptbl := range tables { - tableID, histID, isIndex := ptbl.GetInt64(0), int64(-1), int64(-1) - for { - // fetch at most 100000 rows each time to avoid OOM - const sql = "select table_id, hist_id, is_index, feedback from mysql.stats_feedback where table_id = %? and is_index >= %? and hist_id > %? order by is_index, hist_id limit 100000" - rows, _, err := h.execRestrictedSQL(ctx, sql, tableID, histID, isIndex) - if err != nil { - return errors.Trace(err) - } - if len(rows) == 0 { - break - } - startIdx := 0 - for i, row := range rows { - if row.GetInt64(1) != histID || row.GetInt64(2) != isIndex { - if i > 0 { - if err = h.handleSingleHistogramUpdate(is, rows[startIdx:i]); err != nil { - return errors.Trace(err) - } - } - histID, isIndex = row.GetInt64(1), row.GetInt64(2) - startIdx = i - } - } - if err = h.handleSingleHistogramUpdate(is, rows[startIdx:]); err != nil { - return errors.Trace(err) - } - } - } - return nil -} - -// handleSingleHistogramUpdate updates the Histogram and CM Sketch using these feedbacks. All the feedbacks for -// the same index or column are gathered in `rows`. -func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []chunk.Row) (err error) { - physicalTableID, histID, isIndex := rows[0].GetInt64(0), rows[0].GetInt64(1), rows[0].GetInt64(2) - defer func() { - if err == nil { - err = errors.Trace(h.deleteOutdatedFeedback(physicalTableID, histID, isIndex)) - } - }() - table, ok := h.getTableByPhysicalID(is, physicalTableID) - // The table has been deleted. - if !ok { - return nil - } - var tbl *statistics.Table - // feedback for partition is not ready - if table.Meta().GetPartitionInfo() != nil { - return nil - } - tbl = h.GetTableStats(table.Meta()) - var cms *statistics.CMSketch - var hist *statistics.Histogram - var topN *statistics.TopN - var statsVer int64 = statistics.Version1 - if isIndex == 1 { - idx, ok := tbl.Indices[histID] - statsVer = idx.GetStatsVer() - if statsVer >= 2 { - logutil.BgLogger().Warn("Feedback is discarded because statistics on this table is version 2, which is incompatible with feedback. "+ - "Please consider setting feedback-probability to 0.0 in config file to disable query feedback.", - zap.String("category", "stats"), zap.Int64("table_id", physicalTableID), zap.Int64("hist_id", histID), zap.Int64("is_index", isIndex)) - return err - } - if ok && idx.Histogram.Len() > 0 { - statsVer = idx.GetStatsVer() - idxHist := idx.Histogram - hist = &idxHist - cms = idx.CMSketch.Copy() - topN = idx.TopN.Copy() - } - } else { - col, ok := tbl.Columns[histID] - if ok && col.GetStatsVer() >= 2 { - logutil.BgLogger().Warn("Feedback is discarded because statistics on this table is version 2, which is incompatible with feedback. "+ - "Please consider setting feedback-probability to 0.0 in config file to disable query feedback.", - zap.String("category", "stats"), zap.Int64("table_id", physicalTableID), zap.Int64("hist_id", histID), zap.Int64("is_index", isIndex)) - return err - } - if ok && col.Histogram.Len() > 0 { - colHist := col.Histogram - hist = &colHist - } - } - // The column or index has been deleted. - if hist == nil { - return nil - } - q := &statistics.QueryFeedback{} - for _, row := range rows { - err1 := statistics.DecodeFeedback(row.GetBytes(3), q, cms, topN, hist.Tp) - if err1 != nil { - logutil.BgLogger().Debug("decode feedback failed", zap.Error(err1)) - } - } - err = h.dumpStatsUpdateToKV(physicalTableID, isIndex, q, hist, cms, topN, statsVer) - return errors.Trace(err) -} - -func (h *Handle) deleteOutdatedFeedback(tableID, histID, isIndex int64) error { - h.mu.Lock() - defer h.mu.Unlock() - hasData := true - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) - for hasData { - sql := "delete from mysql.stats_feedback where table_id = %? and hist_id = %? and is_index = %? limit 10000" - _, err := h.mu.ctx.(sqlexec.SQLExecutor).ExecuteInternal(ctx, sql, tableID, histID, isIndex) - if err != nil { - return errors.Trace(err) - } - hasData = h.mu.ctx.GetSessionVars().StmtCtx.AffectedRows() > 0 - } - return nil -} - -func (h *Handle) dumpStatsUpdateToKV(tableID, isIndex int64, q *statistics.QueryFeedback, hist *statistics.Histogram, cms *statistics.CMSketch, topN *statistics.TopN, statsVersion int64) error { - hist = statistics.UpdateHistogram(hist, q, int(statsVersion)) - // feedback for partition is not ready. - err := h.SaveStatsToStorage(tableID, -1, 0, int(isIndex), hist, cms, topN, int(statsVersion), 0, false, StatsMetaHistorySourceFeedBack) - metrics.UpdateStatsCounter.WithLabelValues(metrics.RetLabel(err)).Inc() - return errors.Trace(err) -} - // DumpColStatsUsageToKV sweeps the whole list, updates the column stats usage map and dumps it to KV. func (h *Handle) DumpColStatsUsageToKV() error { if !variable.EnableColumnTracking.Load() { @@ -1320,351 +957,3 @@ func (h *Handle) execAutoAnalyze(statsVer int, analyzeSnapshot bool, sql string, metrics.AutoAnalyzeCounter.WithLabelValues("succ").Inc() } } - -// formatBuckets formats bucket from lowBkt to highBkt. -func formatBuckets(hg *statistics.Histogram, lowBkt, highBkt, idxCols int) string { - if lowBkt == highBkt { - return hg.BucketToString(lowBkt, idxCols) - } - if lowBkt+1 == highBkt { - return fmt.Sprintf("%s, %s", hg.BucketToString(lowBkt, idxCols), hg.BucketToString(highBkt, idxCols)) - } - // do not care the middle buckets - return fmt.Sprintf("%s, (%d buckets, total count %d), %s", hg.BucketToString(lowBkt, idxCols), - highBkt-lowBkt-1, hg.Buckets[highBkt-1].Count-hg.Buckets[lowBkt].Count, hg.BucketToString(highBkt, idxCols)) -} - -func colRangeToStr(c *statistics.Column, ran *ranger.Range, actual int64, factor float64) string { - lowCount, lowBkt := c.LessRowCountWithBktIdx(nil, ran.LowVal[0]) - highCount, highBkt := c.LessRowCountWithBktIdx(nil, ran.HighVal[0]) - return fmt.Sprintf("range: %s, actual: %d, expected: %d, buckets: {%s}", ran.String(), actual, - int64((highCount-lowCount)*factor), formatBuckets(&c.Histogram, lowBkt, highBkt, 0)) -} - -func logForIndexRange(idx *statistics.Index, ran *ranger.Range, actual int64, factor float64) string { - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - lb, err := codec.EncodeKey(sc, nil, ran.LowVal...) - if err != nil { - return "" - } - rb, err := codec.EncodeKey(sc, nil, ran.HighVal...) - if err != nil { - return "" - } - if idx.CMSketch != nil && bytes.Compare(kv.Key(lb).PrefixNext(), rb) >= 0 { - str, err := types.DatumsToString(ran.LowVal, true) - if err != nil { - return "" - } - return fmt.Sprintf("value: %s, actual: %d, expected: %d", str, actual, int64(float64(idx.QueryBytes(nil, lb))*factor)) - } - l, r := types.NewBytesDatum(lb), types.NewBytesDatum(rb) - lowCount, lowBkt := idx.LessRowCountWithBktIdx(nil, l) - highCount, highBkt := idx.LessRowCountWithBktIdx(nil, r) - return fmt.Sprintf("range: %s, actual: %d, expected: %d, histogram: {%s}", ran.String(), actual, - int64((highCount-lowCount)*factor), formatBuckets(&idx.Histogram, lowBkt, highBkt, len(idx.Info.Columns))) -} - -func logForIndex(prefix string, t *statistics.Table, idx *statistics.Index, ranges []*ranger.Range, actual []int64, factor float64) { - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - if idx.CMSketch == nil || idx.GetStatsVer() != statistics.Version1 { - for i, ran := range ranges { - logutil.BgLogger().Debug(prefix, zap.String("index", idx.Info.Name.O), zap.String("rangeStr", logForIndexRange(idx, ran, actual[i], factor))) - } - return - } - for i, ran := range ranges { - rangePosition := statistics.GetOrdinalOfRangeCond(sc, ran) - // only contains range or equality query - if rangePosition == 0 || rangePosition == len(ran.LowVal) { - logutil.BgLogger().Debug(prefix, zap.String("index", idx.Info.Name.O), zap.String("rangeStr", logForIndexRange(idx, ran, actual[i], factor))) - continue - } - equalityString, err := types.DatumsToString(ran.LowVal[:rangePosition], true) - if err != nil { - continue - } - bytes, err := codec.EncodeKey(sc, nil, ran.LowVal[:rangePosition]...) - if err != nil { - continue - } - equalityCount := idx.QueryBytes(nil, bytes) - rang := ranger.Range{ - LowVal: []types.Datum{ran.LowVal[rangePosition]}, - HighVal: []types.Datum{ran.HighVal[rangePosition]}, - Collators: collate.GetBinaryCollatorSlice(1), - } - colName := idx.Info.Columns[rangePosition].Name.L - // prefer index stats over column stats - if idxHist := t.IndexStartWithColumn(colName); idxHist != nil && idxHist.Histogram.Len() > 0 { - rangeString := logForIndexRange(idxHist, &rang, -1, factor) - logutil.BgLogger().Debug(prefix, zap.String("index", idx.Info.Name.O), zap.Int64("actual", actual[i]), - zap.String("equality", equalityString), zap.Uint64("expected equality", equalityCount), - zap.String("range", rangeString)) - } else if colHist := t.ColumnByName(colName); colHist != nil && colHist.Histogram.Len() > 0 { - err = convertRangeType(&rang, colHist.Tp, time.UTC) - if err == nil { - rangeString := colRangeToStr(colHist, &rang, -1, factor) - logutil.BgLogger().Debug(prefix, zap.String("index", idx.Info.Name.O), zap.Int64("actual", actual[i]), - zap.String("equality", equalityString), zap.Uint64("expected equality", equalityCount), - zap.String("range", rangeString)) - } - } else { - count, err := statistics.GetPseudoRowCountByColumnRanges(sc, float64(t.RealtimeCount), []*ranger.Range{&rang}, 0) - if err == nil { - logutil.BgLogger().Debug(prefix, zap.String("index", idx.Info.Name.O), zap.Int64("actual", actual[i]), - zap.String("equality", equalityString), zap.Uint64("expected equality", equalityCount), - zap.Stringer("range", &rang), zap.Float64("pseudo count", math.Round(count))) - } - } - } -} - -func (h *Handle) logDetailedInfo(q *statistics.QueryFeedback) { - t, ok := h.statsCache.Load().GetFromInternal(q.PhysicalID) - if !ok { - return - } - isIndex := q.Hist.IsIndexHist() - ranges, err := q.DecodeToRanges(isIndex) - if err != nil { - logutil.BgLogger().Debug("decode to ranges failed", zap.Error(err)) - return - } - actual := make([]int64, 0, len(q.Feedback)) - for _, fb := range q.Feedback { - actual = append(actual, fb.Count) - } - logPrefix := fmt.Sprintf("[stats-feedback] %s", t.Name) - if isIndex { - idx := t.Indices[q.Hist.ID] - if idx == nil || idx.Histogram.Len() == 0 { - return - } - logForIndex(logPrefix, t, idx, ranges, actual, idx.GetIncreaseFactor(t.RealtimeCount)) - } else { - c := t.Columns[q.Hist.ID] - if c == nil || c.Histogram.Len() == 0 { - return - } - logForPK(logPrefix, c, ranges, actual, c.GetIncreaseFactor(t.RealtimeCount)) - } -} - -func logForPK(prefix string, c *statistics.Column, ranges []*ranger.Range, actual []int64, factor float64) { - for i, ran := range ranges { - if ran.LowVal[0].GetInt64()+1 >= ran.HighVal[0].GetInt64() { - continue - } - logutil.BgLogger().Debug(prefix, zap.String("column", c.Info.Name.O), zap.String("rangeStr", colRangeToStr(c, ran, actual[i], factor))) - } -} - -// RecalculateExpectCount recalculates the expect row count if the origin row count is estimated by pseudo. Deprecated. -func (h *Handle) RecalculateExpectCount(q *statistics.QueryFeedback, enablePseudoForOutdatedStats bool) error { - t, ok := h.statsCache.Load().GetFromInternal(q.PhysicalID) - if !ok { - return nil - } - tablePseudo := t.Pseudo - if enablePseudoForOutdatedStats { - tablePseudo = t.Pseudo || t.IsOutdated() - } - if !tablePseudo { - return nil - } - isIndex := q.Hist.Tp.GetType() == mysql.TypeBlob - id := q.Hist.ID - if isIndex && (t.Indices[id] == nil || !t.Indices[id].NotAccurate()) { - return nil - } - if !isIndex && (t.Columns[id] == nil || !t.Columns[id].NotAccurate()) { - return nil - } - - se, err := h.pool.Get() - if err != nil { - return err - } - sctx := se.(sessionctx.Context) - timeZone := sctx.GetSessionVars().StmtCtx.TimeZone - defer func() { - sctx.GetSessionVars().StmtCtx.TimeZone = timeZone - h.pool.Put(se) - }() - sctx.GetSessionVars().StmtCtx.TimeZone = time.UTC - - ranges, err := q.DecodeToRanges(isIndex) - if err != nil { - return errors.Trace(err) - } - expected := 0.0 - if isIndex { - idx := t.Indices[id] - expected, err = idx.GetRowCount(sctx, nil, ranges, t.RealtimeCount, t.ModifyCount) - } else { - c := t.Columns[id] - expected, err = c.GetColumnRowCount(sctx, ranges, t.RealtimeCount, t.ModifyCount, true) - } - q.Expected = int64(expected) - return err -} - -func (h *Handle) dumpRangeFeedback(sc *stmtctx.StatementContext, ran *ranger.Range, rangeCount float64, q *statistics.QueryFeedback) error { - lowIsNull := ran.LowVal[0].IsNull() - if q.Tp == statistics.IndexType { - lower, err := codec.EncodeKey(sc, nil, ran.LowVal[0]) - if err != nil { - return errors.Trace(err) - } - upper, err := codec.EncodeKey(sc, nil, ran.HighVal[0]) - if err != nil { - return errors.Trace(err) - } - ran.LowVal[0].SetBytes(lower) - ran.HighVal[0].SetBytes(upper) - } else { - if !statistics.SupportColumnType(q.Hist.Tp) { - return nil - } - if ran.LowVal[0].Kind() == types.KindMinNotNull { - ran.LowVal[0] = types.GetMinValue(q.Hist.Tp) - } - if ran.HighVal[0].Kind() == types.KindMaxValue { - ran.HighVal[0] = types.GetMaxValue(q.Hist.Tp) - } - } - ranges, ok := q.Hist.SplitRange(sc, []*ranger.Range{ran}, q.Tp == statistics.IndexType) - if !ok { - logutil.BgLogger().Debug("type of histogram and ranges mismatch") - return nil - } - counts := make([]float64, 0, len(ranges)) - sum := 0.0 - for i, r := range ranges { - // Though after `SplitRange`, we may have ranges like `[l, r]`, we still use - // `betweenRowCount` to compute the estimation since the ranges of feedback are all in `[l, r)` - // form, that is to say, we ignore the exclusiveness of ranges from `SplitRange` and just use - // its result of boundary values. - count := q.Hist.BetweenRowCount(nil, r.LowVal[0], r.HighVal[0]) - // We have to include `NullCount` of histogram for [l, r) cases where l is null because `betweenRowCount` - // does not include null values of lower bound. - if i == 0 && lowIsNull { - count += float64(q.Hist.NullCount) - } - sum += count - counts = append(counts, count) - } - if sum <= 1 { - return nil - } - // We assume that each part contributes the same error rate. - adjustFactor := rangeCount / sum - for i, r := range ranges { - q.Feedback = append(q.Feedback, statistics.Feedback{Lower: &r.LowVal[0], Upper: &r.HighVal[0], Count: int64(counts[i] * adjustFactor)}) - } - return errors.Trace(h.DumpFeedbackToKV(q)) -} - -func convertRangeType(ran *ranger.Range, ft *types.FieldType, loc *time.Location) error { - err := statistics.ConvertDatumsType(ran.LowVal, ft, loc) - if err != nil { - return err - } - return statistics.ConvertDatumsType(ran.HighVal, ft, loc) -} - -// DumpFeedbackForIndex dumps the feedback for index. Deprecated. -// For queries that contains both equality and range query, we will split them and Update accordingly. -func (h *Handle) DumpFeedbackForIndex(q *statistics.QueryFeedback, t *statistics.Table) error { - idx, ok := t.Indices[q.Hist.ID] - if !ok { - return nil - } - - se, err := h.pool.Get() - if err != nil { - return err - } - sctx := se.(sessionctx.Context) - sc := sctx.GetSessionVars().StmtCtx - timeZone := sc.TimeZone - defer func() { - sctx.GetSessionVars().StmtCtx.TimeZone = timeZone - h.pool.Put(se) - }() - sc.TimeZone = time.UTC - - if idx.CMSketch == nil || idx.GetStatsVer() < statistics.Version1 { - return h.DumpFeedbackToKV(q) - } - ranges, err := q.DecodeToRanges(true) - if err != nil { - logutil.BgLogger().Debug("decode feedback ranges fail", zap.Error(err)) - return nil - } - for i, ran := range ranges { - rangePosition := statistics.GetOrdinalOfRangeCond(sc, ran) - // only contains range or equality query - if rangePosition == 0 || rangePosition == len(ran.LowVal) { - continue - } - bytes, err := codec.EncodeKey(sc, nil, ran.LowVal[:rangePosition]...) - if err != nil { - logutil.BgLogger().Debug("encode keys fail", zap.Error(err)) - continue - } - equalityCount := float64(idx.QueryBytes(nil, bytes)) * idx.GetIncreaseFactor(t.RealtimeCount) - rang := &ranger.Range{ - LowVal: []types.Datum{ran.LowVal[rangePosition]}, - HighVal: []types.Datum{ran.HighVal[rangePosition]}, - Collators: collate.GetBinaryCollatorSlice(1), - } - colName := idx.Info.Columns[rangePosition].Name.L - var rangeCount float64 - rangeFB := &statistics.QueryFeedback{PhysicalID: q.PhysicalID} - // prefer index stats over column stats - if idx := t.IndexStartWithColumn(colName); idx != nil && idx.Histogram.Len() != 0 { - rangeCount, err = t.GetRowCountByIndexRanges(sctx, idx.ID, []*ranger.Range{rang}) - rangeFB.Tp, rangeFB.Hist = statistics.IndexType, &idx.Histogram - } else if col := t.ColumnByName(colName); col != nil && col.Histogram.Len() != 0 { - err = convertRangeType(rang, col.Tp, time.UTC) - if err == nil { - rangeCount, err = t.GetRowCountByColumnRanges(sctx, col.ID, []*ranger.Range{rang}) - rangeFB.Tp, rangeFB.Hist = statistics.ColType, &col.Histogram - } - } else { - continue - } - if err != nil { - logutil.BgLogger().Debug("get row count by ranges fail", zap.Error(err)) - continue - } - - equalityCount, rangeCount = getNewCountForIndex(equalityCount, rangeCount, float64(t.RealtimeCount), float64(q.Feedback[i].Count)) - value := types.NewBytesDatum(bytes) - q.Feedback[i] = statistics.Feedback{Lower: &value, Upper: &value, Count: int64(equalityCount)} - err = h.dumpRangeFeedback(sc, rang, rangeCount, rangeFB) - if err != nil { - logutil.BgLogger().Debug("dump range feedback fail", zap.Error(err)) - continue - } - } - return errors.Trace(h.DumpFeedbackToKV(q)) -} - -// minAdjustFactor is the minimum adjust factor of each index feedback. -// We use it to avoid adjusting too much when the assumption of independence failed. -const minAdjustFactor = 0.7 - -// getNewCountForIndex adjust the estimated `eqCount` and `rangeCount` according to the real count. -// We assumes that `eqCount` and `rangeCount` contribute the same error rate. -func getNewCountForIndex(eqCount, rangeCount, totalCount, realCount float64) (equalityCount, rangeCnt float64) { - estimate := (eqCount / totalCount) * (rangeCount / totalCount) * totalCount - if estimate <= 1 { - return eqCount, rangeCount - } - adjustFactor := math.Sqrt(realCount / estimate) - adjustFactor = math.Max(adjustFactor, minAdjustFactor) - return eqCount * adjustFactor, rangeCount * adjustFactor -} diff --git a/statistics/handle/update_list_test.go b/statistics/handle/update_list_test.go index cbedae29c9bc3..1ee9ce256d672 100644 --- a/statistics/handle/update_list_test.go +++ b/statistics/handle/update_list_test.go @@ -17,7 +17,6 @@ package handle import ( "testing" - "github.com/pingcap/tidb/statistics" "github.com/stretchr/testify/require" ) @@ -25,7 +24,6 @@ func TestInsertAndDelete(t *testing.T) { h := Handle{ listHead: &SessionStatsCollector{mapper: make(tableDeltaMap)}, } - h.feedback.data = statistics.NewQueryFeedbackMap() var items []*SessionStatsCollector for i := 0; i < 5; i++ { items = append(items, h.NewSessionStatsCollector()) diff --git a/statistics/handle/updatetest/BUILD.bazel b/statistics/handle/updatetest/BUILD.bazel index dd36350965019..6419a5e719343 100644 --- a/statistics/handle/updatetest/BUILD.bazel +++ b/statistics/handle/updatetest/BUILD.bazel @@ -8,21 +8,17 @@ go_test( "update_test.go", ], flaky = True, - shard_count = 44, + shard_count = 27, deps = [ - "//metrics", "//parser/model", "//parser/mysql", - "//sessionctx/stmtctx", "//sessionctx/variable", "//statistics", "//statistics/handle", "//statistics/handle/cache", - "//statistics/handle/internal", "//testkit", "//testkit/testsetup", "//types", - "//util/codec", "//util/collate", "//util/ranger", "@com_github_prometheus_client_model//go", diff --git a/statistics/handle/updatetest/update_test.go b/statistics/handle/updatetest/update_test.go index 71849faa605b4..ff2e85259256b 100644 --- a/statistics/handle/updatetest/update_test.go +++ b/statistics/handle/updatetest/update_test.go @@ -22,18 +22,14 @@ import ( "testing" "time" - "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/statistics/handle/cache" - "github.com/pingcap/tidb/statistics/handle/internal" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/ranger" dto "github.com/prometheus/client_model/go" @@ -704,148 +700,6 @@ func TestTableAnalyzed(t *testing.T) { require.True(t, handle.TableAnalyzed(statsTbl)) } -func TestUpdateErrorRate(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - h := dom.StatsHandle() - is := dom.InfoSchema() - h.SetLease(0) - require.NoError(t, h.Update(is)) - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("set @@session.tidb_enable_pseudo_for_outdated_stats = 1") - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") - err := h.HandleDDLEvent(<-h.DDLEventCh()) - require.NoError(t, err) - - testKit.MustExec("insert into t values (1, 3)") - - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t") - - testKit.MustExec("insert into t values (2, 3)") - testKit.MustExec("insert into t values (5, 3)") - testKit.MustExec("insert into t values (8, 3)") - testKit.MustExec("insert into t values (12, 3)") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - is = dom.InfoSchema() - require.NoError(t, h.Update(is)) - - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - tbl := h.GetTableStats(tblInfo) - aID := tblInfo.Columns[0].ID - bID := tblInfo.Indices[0].ID - - // The statistic table is outdated now. - require.True(t, tbl.Columns[aID].NotAccurate()) - - testKit.MustQuery("select * from t where a between 1 and 10") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(is)) - h.UpdateErrorRate(is) - require.NoError(t, h.Update(is)) - tbl = h.GetTableStats(tblInfo) - - // The error rate of this column is not larger than MaxErrorRate now. - require.False(t, tbl.Columns[aID].NotAccurate()) - - require.True(t, tbl.Indices[bID].NotAccurate()) - testKit.MustQuery("select * from t where b between 2 and 10") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(is)) - h.UpdateErrorRate(is) - require.NoError(t, h.Update(is)) - tbl = h.GetTableStats(tblInfo) - require.False(t, tbl.Indices[bID].NotAccurate()) - require.Equal(t, int64(1), tbl.Indices[bID].QueryTotal) - - testKit.MustExec("analyze table t") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.Update(is)) - tbl = h.GetTableStats(tblInfo) - require.Equal(t, int64(0), tbl.Indices[bID].QueryTotal) -} - -func TestUpdatePartitionErrorRate(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - h := dom.StatsHandle() - is := dom.InfoSchema() - h.SetLease(0) - require.NoError(t, h.Update(is)) - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) - testKit.MustExec("create table t (a bigint(64), primary key(a)) partition by range (a) (partition p0 values less than (30))") - err := h.HandleDDLEvent(<-h.DDLEventCh()) - require.NoError(t, err) - - testKit.MustExec("insert into t values (1)") - - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t") - - testKit.MustExec("insert into t values (2)") - testKit.MustExec("insert into t values (5)") - testKit.MustExec("insert into t values (8)") - testKit.MustExec("insert into t values (12)") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - is = dom.InfoSchema() - require.NoError(t, h.Update(is)) - - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - pid := tblInfo.Partition.Definitions[0].ID - tbl := h.GetPartitionStats(tblInfo, pid) - aID := tblInfo.Columns[0].ID - - // The statistic table is outdated now. - require.True(t, tbl.Columns[aID].NotAccurate()) - - testKit.MustQuery("select * from t where a between 1 and 10") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(is)) - h.UpdateErrorRate(is) - require.NoError(t, h.Update(is)) - tbl = h.GetPartitionStats(tblInfo, pid) - - // Feedback will not take effect under partition table. - require.True(t, tbl.Columns[aID].NotAccurate()) -} - func appendBucket(h *statistics.Histogram, l, r int64) { lower, upper := types.NewIntDatum(l), types.NewIntDatum(r) h.AppendBucket(&lower, &upper, 0, 0) @@ -906,232 +760,6 @@ func TestSplitRange(t *testing.T) { } } -func TestQueryFeedback(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") - testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") - testKit.MustExec("analyze table t with 0 topn") - testKit.MustExec("insert into t values (3,4)") - - h := dom.StatsHandle() - oriProbability := statistics.FeedbackProbability.Load() - oriNumber := statistics.MaxNumberOfRanges - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - statistics.MaxNumberOfRanges = oriNumber - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - tests := []struct { - sql string - hist string - idxCols int - }{ - { - // test primary key feedback - sql: "select * from t where t.a <= 5 order by a desc", - hist: "column:1 ndv:4 totColSize:0\n" + - "num: 1 lower_bound: -9223372036854775808 upper_bound: 2 repeats: 0 ndv: 0\n" + - "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 0\n" + - "num: 1 lower_bound: 4 upper_bound: 4 repeats: 1 ndv: 0", - idxCols: 0, - }, - { - // test index feedback by double read - sql: "select * from t use index(idx) where t.b <= 5", - hist: "index:1 ndv:2\n" + - "num: 3 lower_bound: -inf upper_bound: 5 repeats: 0 ndv: 0\n" + - "num: 1 lower_bound: 5 upper_bound: 5 repeats: 1 ndv: 0", - idxCols: 1, - }, - { - // test index feedback by single read - sql: "select b from t use index(idx) where t.b <= 5", - hist: "index:1 ndv:2\n" + - "num: 3 lower_bound: -inf upper_bound: 5 repeats: 0 ndv: 0\n" + - "num: 1 lower_bound: 5 upper_bound: 5 repeats: 1 ndv: 0", - idxCols: 1, - }, - } - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - for i, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, err) - require.NoError(t, h.Update(is)) - tblInfo := table.Meta() - tbl := h.GetTableStats(tblInfo) - if test.idxCols == 0 { - require.Equal(t, tests[i].hist, tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) - } else { - require.Equal(t, tests[i].hist, tbl.Indices[tblInfo.Indices[0].ID].ToString(1)) - } - } - - // Feedback from limit executor may not be accurate. - testKit.MustQuery("select * from t where t.a <= 5 limit 1") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - feedback := h.GetQueryFeedback() - require.Equal(t, 0, feedback.Size) - - // Test only collect for max number of Ranges. - statistics.MaxNumberOfRanges = 0 - for _, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - feedback := h.GetQueryFeedback() - require.Equal(t, 0, feedback.Size) - } - - // Test collect feedback by probability. - statistics.FeedbackProbability.Store(0) - statistics.MaxNumberOfRanges = oriNumber - for _, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - feedback := h.GetQueryFeedback() - require.Equal(t, 0, feedback.Size) - } - - // Test that after drop stats, the feedback won't cause panic. - statistics.FeedbackProbability.Store(1) - for _, test := range tests { - testKit.MustQuery(test.sql) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - testKit.MustExec("drop stats t") - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - - // Test that the outdated feedback won't cause panic. - testKit.MustExec("analyze table t") - for _, test := range tests { - testKit.MustQuery(test.sql) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - testKit.MustExec("drop table t") - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) -} - -func TestQueryFeedbackForPartition(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) - testKit.MustExec(`create table t (a bigint(64), b bigint(64), primary key(a), index idx(b)) - partition by range (a) ( - partition p0 values less than (3), - partition p1 values less than (6))`) - testKit.MustExec("insert into t values (1,2),(2,2),(3,4),(4,1),(5,6)") - testKit.MustExec("analyze table t") - - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - h := dom.StatsHandle() - // Feedback will not take effect under partition table. - tests := []struct { - sql string - hist string - idxCols int - }{ - { - // test primary key feedback - sql: "select * from t where t.a <= 5", - hist: "column:1 ndv:2 totColSize:2\n" + - "num: 1 lower_bound: 1 upper_bound: 1 repeats: 1 ndv: 0\n" + - "num: 1 lower_bound: 2 upper_bound: 2 repeats: 1 ndv: 0", - idxCols: 0, - }, - { - // test index feedback by double read - sql: "select * from t use index(idx) where t.b <= 5", - hist: "index:1 ndv:1\n" + - "num: 2 lower_bound: 2 upper_bound: 2 repeats: 2 ndv: 0", - idxCols: 1, - }, - { - // test index feedback by single read - sql: "select b from t use index(idx) where t.b <= 5", - hist: "index:1 ndv:1\n" + - "num: 2 lower_bound: 2 upper_bound: 2 repeats: 2 ndv: 0", - idxCols: 1, - }, - } - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - pi := tblInfo.GetPartitionInfo() - require.NotNil(t, pi) - - // This test will check the result of partition p0. - var pid int64 - for _, def := range pi.Definitions { - if def.Name.L == "p0" { - pid = def.ID - break - } - } - - for i, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, err) - require.NoError(t, h.Update(is)) - tbl := h.GetPartitionStats(tblInfo, pid) - if test.idxCols == 0 { - require.Equal(t, tests[i].hist, tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) - } else { - require.Equal(t, tests[i].hist, tbl.Indices[tblInfo.Indices[0].ID].ToString(1)) - } - } - testKit.MustExec("drop table t") -} - -func TestUpdateSystemTable(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("create table t (a int, b int)") - testKit.MustExec("insert into t values (1,2)") - testKit.MustExec("analyze table t") - testKit.MustExec("analyze table mysql.stats_histograms") - h := dom.StatsHandle() - require.NoError(t, h.Update(dom.InfoSchema())) - feedback := h.GetQueryFeedback() - // We may have query feedback for system tables, but we do not need to store them. - require.Equal(t, 0, feedback.Size) -} - func TestOutOfOrderUpdate(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) testKit := testkit.NewTestKit(t, store) @@ -1166,214 +794,6 @@ func TestOutOfOrderUpdate(t *testing.T) { testKit.MustQuery(fmt.Sprintf("select count from mysql.stats_meta where table_id = %d", tableInfo.ID)).Check(testkit.Rows("3")) } -func TestUpdateStatsByLocalFeedback(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) - testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") - testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") - testKit.MustExec("analyze table t with 0 topn") - testKit.MustExec("insert into t values (3,5)") - h := dom.StatsHandle() - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - oriNumber := statistics.MaxNumberOfRanges - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - statistics.MaxNumberOfRanges = oriNumber - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - - tblInfo := table.Meta() - h.GetTableStats(tblInfo) - - testKit.MustQuery("select * from t use index(idx) where b <= 5") - testKit.MustQuery("select * from t where a > 1") - testKit.MustQuery("select * from t use index(idx) where b = 5") - - h.UpdateStatsByLocalFeedback(dom.InfoSchema()) - tbl := h.GetTableStats(tblInfo) - - require.Equal(t, "column:1 ndv:3 totColSize:0\n"+ - "num: 1 lower_bound: 1 upper_bound: 1 repeats: 1 ndv: 0\n"+ - "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 0\n"+ - "num: 1 lower_bound: 4 upper_bound: 9223372036854775807 repeats: 0 ndv: 0", tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) - sc := &stmtctx.StatementContext{TimeZone: time.Local} - low, err := codec.EncodeKey(sc, nil, types.NewIntDatum(5)) - require.NoError(t, err) - - require.Equal(t, uint64(2), tbl.Indices[tblInfo.Indices[0].ID].CMSketch.QueryBytes(low)) - - require.Equal(t, "index:1 ndv:2\n"+ - "num: 2 lower_bound: -inf upper_bound: 5 repeats: 0 ndv: 0\n"+ - "num: 1 lower_bound: 5 upper_bound: 5 repeats: 1 ndv: 0", tbl.Indices[tblInfo.Indices[0].ID].ToString(1)) - - // Test that it won't cause panic after update. - testKit.MustQuery("select * from t use index(idx) where b > 0") - - // Test that after drop stats, it won't cause panic. - testKit.MustExec("drop stats t") - h.UpdateStatsByLocalFeedback(dom.InfoSchema()) -} - -func TestUpdatePartitionStatsByLocalFeedback(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) - testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a)) partition by range (a) (partition p0 values less than (6))") - testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") - testKit.MustExec("analyze table t") - testKit.MustExec("insert into t values (3,5)") - h := dom.StatsHandle() - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - - testKit.MustQuery("select * from t where a > 1").Check(testkit.Rows("2 2", "3 5", "4 5")) - - h.UpdateStatsByLocalFeedback(dom.InfoSchema()) - - tblInfo := table.Meta() - pid := tblInfo.Partition.Definitions[0].ID - tbl := h.GetPartitionStats(tblInfo, pid) - - // Feedback will not take effect under partition table. - require.Equal(t, "column:1 ndv:3 totColSize:0\n"+ - "num: 1 lower_bound: 1 upper_bound: 1 repeats: 1 ndv: 0\n"+ - "num: 1 lower_bound: 2 upper_bound: 2 repeats: 1 ndv: 0\n"+ - "num: 1 lower_bound: 4 upper_bound: 4 repeats: 1 ndv: 0", tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) -} - -func TestFeedbackWithStatsVer2(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("set global tidb_analyze_version = 1") - testKit.MustExec("set @@tidb_analyze_version = 1") - - oriProbability := statistics.FeedbackProbability.Load() - oriNumber := statistics.MaxNumberOfRanges - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - statistics.MaxNumberOfRanges = oriNumber - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - // Case 1: You can't set tidb_analyze_version to 2 if feedback is enabled. - statistics.FeedbackProbability.Store(1) - testKit.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("1")) - testKit.MustExec("set @@tidb_analyze_version = 2") - testKit.MustQuery("show warnings").Check(testkit.Rows(`Error 1105 variable tidb_analyze_version not updated because analyze version 2 is incompatible with query feedback. Please consider setting feedback-probability to 0.0 in config file to disable query feedback`)) - testKit.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("1")) - - // Case 2: Feedback wouldn't be applied on version 2 statistics. - statistics.FeedbackProbability.Store(0) - testKit.MustExec("set @@tidb_analyze_version = 2") - testKit.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("2")) - testKit.MustExec("create table t (a bigint(64), b bigint(64), index idx(b))") - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t values (1,2),(2,2),(4,5),(2,3),(3,4)") - } - testKit.MustExec("analyze table t with 0 topn") - h := dom.StatsHandle() - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - testKit.MustExec("analyze table t") - err = h.Update(dom.InfoSchema()) - require.NoError(t, err) - statsTblBefore := h.GetTableStats(tblInfo) - statistics.FeedbackProbability.Store(1) - // make the statistics inaccurate. - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t values (3,4), (3,4), (3,4), (3,4), (3,4)") - } - // trigger feedback - testKit.MustExec("select * from t where t.a <= 5 order by a desc") - testKit.MustExec("select b from t use index(idx) where t.b <= 5") - - h.UpdateStatsByLocalFeedback(dom.InfoSchema()) - err = h.DumpStatsFeedbackToKV() - require.NoError(t, err) - err = h.HandleUpdateStats(dom.InfoSchema()) - require.NoError(t, err) - statsTblAfter := h.GetTableStats(tblInfo) - // assert that statistics not changed - internal.AssertTableEqual(t, statsTblBefore, statsTblAfter) - - // Case 3: Feedback is still effective on version 1 statistics. - testKit.MustExec("set tidb_analyze_version = 1") - testKit.MustExec("create table t1 (a bigint(64), b bigint(64), index idx(b))") - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t1 values (1,2),(2,2),(4,5),(2,3),(3,4)") - } - testKit.MustExec("analyze table t1 with 0 topn") - // make the statistics inaccurate. - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t1 values (3,4), (3,4), (3,4), (3,4), (3,4)") - } - is = dom.InfoSchema() - table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - require.NoError(t, err) - tblInfo = table.Meta() - statsTblBefore = h.GetTableStats(tblInfo) - // trigger feedback - testKit.MustExec("select b from t1 use index(idx) where t1.b <= 5") - - h.UpdateStatsByLocalFeedback(dom.InfoSchema()) - err = h.DumpStatsFeedbackToKV() - require.NoError(t, err) - err = h.HandleUpdateStats(dom.InfoSchema()) - require.NoError(t, err) - statsTblAfter = h.GetTableStats(tblInfo) - // assert that statistics changed(feedback worked) - require.False(t, statistics.HistogramEqual(&statsTblBefore.Indices[1].Histogram, &statsTblAfter.Indices[1].Histogram, false)) - - // Case 4: When existing version 1 stats + tidb_analyze_version=2 + feedback enabled, explicitly running `analyze table` still results in version 1 stats. - statistics.FeedbackProbability.Store(0) - testKit.MustExec("set tidb_analyze_version = 2") - statistics.FeedbackProbability.Store(1) - testKit.MustExec("analyze table t1 with 0 topn") - testKit.MustQuery("show warnings").Check(testkit.Rows( - "Warning 1105 Use analyze version 1 on table `t1` because this table already has version 1 statistics and query feedback is also enabled." + - " If you want to switch to version 2 statistics, please first disable query feedback by setting feedback-probability to 0.0 in the config file.")) - testKit.MustQuery(fmt.Sprintf("select stats_ver from mysql.stats_histograms where table_id = %d", tblInfo.ID)).Check(testkit.Rows("1", "1", "1")) - - testKit.MustExec("set global tidb_analyze_version = 1") -} - func TestNeedAnalyzeTable(t *testing.T) { columns := map[int64]*statistics.Column{} columns[1] = &statistics.Column{StatsVer: statistics.Version2} @@ -1456,452 +876,6 @@ func TestNeedAnalyzeTable(t *testing.T) { } } -func TestIndexQueryFeedback(t *testing.T) { - t.Skip("support update the topn of index equal conditions") - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - - oriProbability := statistics.FeedbackProbability.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - }() - statistics.FeedbackProbability.Store(1) - - testKit.MustExec("use test") - testKit.MustExec("create table t (a bigint(64), b bigint(64), c bigint(64), d float, e double, f decimal(17,2), " + - "g time, h date, index idx_b(b), index idx_ab(a,b), index idx_ac(a,c), index idx_ad(a, d), index idx_ae(a, e), index idx_af(a, f)," + - " index idx_ag(a, g), index idx_ah(a, h))") - for i := 0; i < 20; i++ { - testKit.MustExec(fmt.Sprintf(`insert into t values (1, %d, %d, %d, %d, %d, %d, "%s")`, i, i, i, i, i, i, fmt.Sprintf("1000-01-%02d", i+1))) - } - h := dom.StatsHandle() - err := h.HandleDDLEvent(<-h.DDLEventCh()) - require.NoError(t, err) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t with 3 buckets") - for i := 0; i < 20; i++ { - testKit.MustExec(fmt.Sprintf(`insert into t values (1, %d, %d, %d, %d, %d, %d, "%s")`, i, i, i, i, i, i, fmt.Sprintf("1000-01-%02d", i+1))) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - is := dom.InfoSchema() - require.NoError(t, h.Update(is)) - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - tests := []struct { - sql string - hist string - idxCols int - rangeID int64 - idxID int64 - eqCount uint32 - }{ - { - sql: "select * from t use index(idx_ab) where a = 1 and b < 21", - hist: "index:1 ndv:20\n" + - "num: 16 lower_bound: -inf upper_bound: 7 repeats: 0\n" + - "num: 16 lower_bound: 8 upper_bound: 15 repeats: 0\n" + - "num: 9 lower_bound: 16 upper_bound: 21 repeats: 0", - rangeID: tblInfo.Indices[0].ID, - idxID: tblInfo.Indices[1].ID, - idxCols: 1, - eqCount: 32, - }, - { - sql: "select * from t use index(idx_ac) where a = 1 and c < 21", - hist: "column:3 ndv:20 totColSize:40\n" + - "num: 13 lower_bound: -9223372036854775808 upper_bound: 6 repeats: 0\n" + - "num: 13 lower_bound: 7 upper_bound: 13 repeats: 0\n" + - "num: 12 lower_bound: 14 upper_bound: 21 repeats: 0", - rangeID: tblInfo.Columns[2].ID, - idxID: tblInfo.Indices[2].ID, - idxCols: 0, - eqCount: 32, - }, - { - sql: "select * from t use index(idx_ad) where a = 1 and d < 21", - hist: "column:4 ndv:20 totColSize:320\n" + - "num: 13 lower_bound: -10000000000000 upper_bound: 6 repeats: 0\n" + - "num: 12 lower_bound: 7 upper_bound: 13 repeats: 0\n" + - "num: 10 lower_bound: 14 upper_bound: 21 repeats: 0", - rangeID: tblInfo.Columns[3].ID, - idxID: tblInfo.Indices[3].ID, - idxCols: 0, - eqCount: 32, - }, - { - sql: "select * from t use index(idx_ae) where a = 1 and e < 21", - hist: "column:5 ndv:20 totColSize:320\n" + - "num: 13 lower_bound: -100000000000000000000000 upper_bound: 6 repeats: 0\n" + - "num: 12 lower_bound: 7 upper_bound: 13 repeats: 0\n" + - "num: 10 lower_bound: 14 upper_bound: 21 repeats: 0", - rangeID: tblInfo.Columns[4].ID, - idxID: tblInfo.Indices[4].ID, - idxCols: 0, - eqCount: 32, - }, - { - sql: "select * from t use index(idx_af) where a = 1 and f < 21", - hist: "column:6 ndv:20 totColSize:400\n" + - "num: 13 lower_bound: -999999999999999.99 upper_bound: 6.00 repeats: 0\n" + - "num: 12 lower_bound: 7.00 upper_bound: 13.00 repeats: 0\n" + - "num: 10 lower_bound: 14.00 upper_bound: 21.00 repeats: 0", - rangeID: tblInfo.Columns[5].ID, - idxID: tblInfo.Indices[5].ID, - idxCols: 0, - eqCount: 32, - }, - { - sql: "select * from t use index(idx_ag) where a = 1 and g < 21", - hist: "column:7 ndv:20 totColSize:196\n" + - "num: 13 lower_bound: -838:59:59 upper_bound: 00:00:06 repeats: 0\n" + - "num: 12 lower_bound: 00:00:07 upper_bound: 00:00:13 repeats: 0\n" + - "num: 10 lower_bound: 00:00:14 upper_bound: 00:00:21 repeats: 0", - rangeID: tblInfo.Columns[6].ID, - idxID: tblInfo.Indices[6].ID, - idxCols: 0, - eqCount: 30, - }, - { - sql: `select * from t use index(idx_ah) where a = 1 and h < "1000-01-21"`, - hist: "column:8 ndv:20 totColSize:360\n" + - "num: 13 lower_bound: 1000-01-01 upper_bound: 1000-01-07 repeats: 0\n" + - "num: 12 lower_bound: 1000-01-08 upper_bound: 1000-01-14 repeats: 0\n" + - "num: 10 lower_bound: 1000-01-15 upper_bound: 1000-01-21 repeats: 0", - rangeID: tblInfo.Columns[7].ID, - idxID: tblInfo.Indices[7].ID, - idxCols: 0, - eqCount: 32, - }, - } - for i, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, h.Update(is)) - tbl := h.GetTableStats(tblInfo) - if test.idxCols == 0 { - require.Equal(t, tests[i].hist, tbl.Columns[test.rangeID].ToString(0)) - } else { - require.Equal(t, tests[i].hist, tbl.Indices[test.rangeID].ToString(1)) - } - val, err := codec.EncodeKey(testKit.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) - require.NoError(t, err) - require.Equal(t, uint64(test.eqCount), tbl.Indices[test.idxID].CMSketch.QueryBytes(val)) - } -} - -func TestIndexQueryFeedback4TopN(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec("create table t (a bigint(64), index idx(a))") - for i := 0; i < 20; i++ { - testKit.MustExec(`insert into t values (1)`) - } - h := dom.StatsHandle() - err := h.HandleDDLEvent(<-h.DDLEventCh()) - require.NoError(t, err) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("set @@session.tidb_analyze_version = 1") - testKit.MustExec("set @@tidb_enable_fast_analyze = 1") - testKit.MustExec("analyze table t with 3 buckets") - for i := 0; i < 20; i++ { - testKit.MustExec(`insert into t values (1)`) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - is := dom.InfoSchema() - require.NoError(t, h.Update(is)) - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - - testKit.MustQuery("select * from t use index(idx) where a = 1") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, h.Update(is)) - tbl := h.GetTableStats(tblInfo) - val, err := codec.EncodeKey(testKit.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) - require.NoError(t, err) - require.Equal(t, uint64(40), tbl.Indices[1].CMSketch.QueryBytes(val)) -} - -func TestAbnormalIndexFeedback(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("create table t (a bigint(64), b bigint(64), index idx_ab(a,b))") - for i := 0; i < 20; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i/5, i)) - } - testKit.MustExec("set @@session.tidb_analyze_version = 1") - testKit.MustExec("analyze table t with 3 buckets, 0 topn") - testKit.MustExec("delete from t where a = 1") - testKit.MustExec("delete from t where b > 10") - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - h := dom.StatsHandle() - tests := []struct { - sql string - hist string - rangeID int64 - idxID int64 - eqCount uint32 - }{ - { - // The real count of `a = 1` is 0. - sql: "select * from t where a = 1 and b < 21", - hist: "column:2 ndv:20 totColSize:20\n" + - "num: 5 lower_bound: -9223372036854775808 upper_bound: 7 repeats: 0 ndv: 0\n" + - "num: 4 lower_bound: 7 upper_bound: 14 repeats: 0 ndv: 0\n" + - "num: 4 lower_bound: 14 upper_bound: 21 repeats: 0 ndv: 0", - rangeID: tblInfo.Columns[1].ID, - idxID: tblInfo.Indices[0].ID, - eqCount: 3, - }, - { - // The real count of `b > 10` is 0. - sql: "select * from t where a = 2 and b > 10", - hist: "column:2 ndv:20 totColSize:20\n" + - "num: 5 lower_bound: -9223372036854775808 upper_bound: 7 repeats: 0 ndv: 0\n" + - "num: 6 lower_bound: 7 upper_bound: 14 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 14 upper_bound: 9223372036854775807 repeats: 0 ndv: 0", - rangeID: tblInfo.Columns[1].ID, - idxID: tblInfo.Indices[0].ID, - eqCount: 3, - }, - } - for i, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, h.Update(is)) - tbl := h.GetTableStats(tblInfo) - require.Equal(t, tests[i].hist, tbl.Columns[test.rangeID].ToString(0)) - val, err := codec.EncodeKey(testKit.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) - require.NoError(t, err) - require.Equal(t, uint64(test.eqCount), tbl.Indices[test.idxID].CMSketch.QueryBytes(val)) - } -} - -func TestFeedbackRanges(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - h := dom.StatsHandle() - oriProbability := statistics.FeedbackProbability.Load() - oriNumber := statistics.MaxNumberOfRanges - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - statistics.MaxNumberOfRanges = oriNumber - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("create table t (a tinyint, b tinyint, primary key(a), index idx(a, b))") - for i := 0; i < 20; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) - } - err := h.HandleDDLEvent(<-h.DDLEventCh()) - require.NoError(t, err) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("set @@session.tidb_analyze_version=1") - testKit.MustExec("analyze table t with 3 buckets") - for i := 30; i < 40; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - tests := []struct { - sql string - hist string - colID int64 - }{ - { - sql: "select * from t where a <= 50 or (a > 130 and a < 140)", - hist: "column:1 ndv:30 totColSize:0\n" + - "num: 8 lower_bound: -128 upper_bound: 8 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 8 upper_bound: 16 repeats: 0 ndv: 0\n" + - "num: 14 lower_bound: 16 upper_bound: 50 repeats: 0 ndv: 0", - colID: 1, - }, - { - sql: "select * from t where a >= 10", - hist: "column:1 ndv:30 totColSize:0\n" + - "num: 8 lower_bound: -128 upper_bound: 8 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 8 upper_bound: 16 repeats: 0 ndv: 0\n" + - "num: 14 lower_bound: 16 upper_bound: 127 repeats: 0 ndv: 0", - colID: 1, - }, - { - sql: "select * from t use index(idx) where a = 1 and (b <= 50 or (b > 130 and b < 140))", - hist: "column:2 ndv:20 totColSize:30\n" + - "num: 8 lower_bound: -128 upper_bound: 7 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 7 upper_bound: 14 repeats: 0 ndv: 0\n" + - "num: 7 lower_bound: 14 upper_bound: 51 repeats: 0 ndv: 0", - colID: 2, - }, - } - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - for _, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, err) - require.NoError(t, h.Update(is)) - tblInfo := table.Meta() - tbl := h.GetTableStats(tblInfo) - require.Equal(t, test.hist, tbl.Columns[test.colID].ToString(0)) - } -} - -func TestUnsignedFeedbackRanges(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - h := dom.StatsHandle() - - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - oriNumber := statistics.MaxNumberOfRanges - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - statistics.MaxNumberOfRanges = oriNumber - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec("create table t (a tinyint unsigned, primary key(a))") - testKit.MustExec("create table t1 (a bigint unsigned, primary key(a))") - for i := 0; i < 20; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d)", i)) - testKit.MustExec(fmt.Sprintf("insert into t1 values (%d)", i)) - } - err := h.HandleDDLEvent(<-h.DDLEventCh()) - require.NoError(t, err) - err = h.HandleDDLEvent(<-h.DDLEventCh()) - require.NoError(t, err) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t, t1 with 3 buckets") - for i := 30; i < 40; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d)", i)) - testKit.MustExec(fmt.Sprintf("insert into t1 values (%d)", i)) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - tests := []struct { - sql string - hist string - tblName string - }{ - { - sql: "select * from t where a <= 50", - hist: "column:1 ndv:30 totColSize:10\n" + - "num: 8 lower_bound: 0 upper_bound: 8 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 8 upper_bound: 16 repeats: 0 ndv: 0\n" + - "num: 14 lower_bound: 16 upper_bound: 50 repeats: 0 ndv: 0", - tblName: "t", - }, - { - sql: "select count(*) from t", - hist: "column:1 ndv:30 totColSize:10\n" + - "num: 8 lower_bound: 0 upper_bound: 8 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 8 upper_bound: 16 repeats: 0 ndv: 0\n" + - "num: 14 lower_bound: 16 upper_bound: 255 repeats: 0 ndv: 0", - tblName: "t", - }, - { - sql: "select * from t1 where a <= 50", - hist: "column:1 ndv:30 totColSize:10\n" + - "num: 8 lower_bound: 0 upper_bound: 8 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 8 upper_bound: 16 repeats: 0 ndv: 0\n" + - "num: 14 lower_bound: 16 upper_bound: 50 repeats: 0 ndv: 0", - tblName: "t1", - }, - { - sql: "select count(*) from t1", - hist: "column:1 ndv:30 totColSize:10\n" + - "num: 8 lower_bound: 0 upper_bound: 8 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 8 upper_bound: 16 repeats: 0 ndv: 0\n" + - "num: 14 lower_bound: 16 upper_bound: 18446744073709551615 repeats: 0 ndv: 0", - tblName: "t1", - }, - } - is := dom.InfoSchema() - require.NoError(t, h.Update(is)) - for _, test := range tests { - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr(test.tblName)) - require.NoError(t, err) - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, err) - require.NoError(t, h.Update(is)) - tblInfo := table.Meta() - tbl := h.GetTableStats(tblInfo) - require.Equal(t, test.hist, tbl.Columns[1].ToString(0)) - } -} - func TestLoadHistCorrelation(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) testKit := testkit.NewTestKit(t, store) @@ -1927,36 +901,6 @@ func TestLoadHistCorrelation(t *testing.T) { require.Equal(t, "1", result.Rows()[0][9]) } -func TestDeleteUpdateFeedback(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - - oriProbability := statistics.FeedbackProbability.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - }() - statistics.FeedbackProbability.Store(1) - - h := dom.StatsHandle() - testKit.MustExec("use test") - testKit.MustExec("create table t (a bigint(64), b bigint(64), index idx_ab(a,b))") - for i := 0; i < 20; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i/5, i)) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t with 3 buckets") - - testKit.MustExec("delete from t where a = 1") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.Equal(t, 0, h.GetQueryFeedback().Size) - testKit.MustExec("update t set a = 6 where a = 2") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.Equal(t, 0, h.GetQueryFeedback().Size) - testKit.MustExec("explain analyze delete from t where a = 3") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.Equal(t, 0, h.GetQueryFeedback().Size) -} - func BenchmarkHandleAutoAnalyze(b *testing.B) { store, dom := testkit.CreateMockStoreAndDomain(b) testKit := testkit.NewTestKit(b, store) @@ -1974,62 +918,6 @@ func subtraction(newMetric *dto.Metric, oldMetric *dto.Metric) int { return int(*(newMetric.Counter.Value) - *(oldMetric.Counter.Value)) } -func TestDisableFeedback(t *testing.T) { - store := testkit.CreateMockStore(t) - testKit := testkit.NewTestKit(t, store) - - oriProbability := statistics.FeedbackProbability.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - }() - statistics.FeedbackProbability.Store(0.0) - oldNum := &dto.Metric{} - err := metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(oldNum) - require.NoError(t, err) - testKit.MustExec("use test") - testKit.MustExec("create table t (a int, b int, index idx_a(a))") - testKit.MustExec("insert into t values (1, 1), (2, 2), (3, 3), (5, 5)") - testKit.MustExec("analyze table t with 0 topn") - for i := 0; i < 20; i++ { - testKit.MustQuery("select /*+ use_index(t, idx_a) */ * from t where a < 4") - } - - newNum := &dto.Metric{} - err = metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(newNum) - require.NoError(t, err) - require.Equal(t, 0, subtraction(newNum, oldNum)) -} - -func TestFeedbackCounter(t *testing.T) { - store := testkit.CreateMockStore(t) - testKit := testkit.NewTestKit(t, store) - - oriProbability := statistics.FeedbackProbability.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - }() - statistics.FeedbackProbability.Store(1) - oldNum := &dto.Metric{} - err := metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(oldNum) - require.NoError(t, err) - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("create table t (a int, b int, index idx_a(a))") - testKit.MustExec("insert into t values (1, 1), (2, 2), (3, 3), (5, 5)") - testKit.MustExec("analyze table t with 0 topn") - for i := 0; i < 20; i++ { - testKit.MustQuery("select /*+ use_index(t, idx_a) */ * from t where a < 4") - } - - newNum := &dto.Metric{} - err = metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(newNum) - require.NoError(t, err) - require.Equal(t, 20, subtraction(newNum, oldNum)) -} - func TestMergeTopN(t *testing.T) { // Move this test to here to avoid race test. tests := []struct { @@ -2475,98 +1363,6 @@ func TestStatsLockUnlockForAutoAnalyze(t *testing.T) { require.Equal(t, int64(15), tblStats2.RealtimeCount) } -func TestStatsLockForFeedback(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") - testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") - testKit.MustExec("analyze table t with 0 topn") - testKit.MustExec("insert into t values (3,4)") - for i := 5; i < 20; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values(%d, %d)", i, i+1)) - } - - h := dom.StatsHandle() - oriProbability := statistics.FeedbackProbability.Load() - oriNumber := statistics.MaxNumberOfRanges - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - statistics.MaxNumberOfRanges = oriNumber - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - tests := []struct { - sql string - hist string - }{ - { - // test primary key feedback - sql: "select * from t where t.a <= 4 order by a desc", - hist: "column:1 ndv:4 totColSize:0\n" + - "num: 1 lower_bound: -9223372036854775808 upper_bound: 2 repeats: 0 ndv: 0\n" + - "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 0\n" + - "num: 1 lower_bound: 4 upper_bound: 4 repeats: 1 ndv: 0", - }, - //run 1st sql after table locked, hist should not changed - { - sql: "select * from t where t.a <= 8 order by a desc", - hist: "column:1 ndv:4 totColSize:0\n" + - "num: 1 lower_bound: -9223372036854775808 upper_bound: 2 repeats: 0 ndv: 0\n" + - "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 0\n" + - "num: 1 lower_bound: 4 upper_bound: 4 repeats: 1 ndv: 0", - }, - //run 2nd sql after table unlocked, hist should not changed - { - sql: "select * from t where t.a <= 12 order by a desc", - hist: "column:1 ndv:12 totColSize:0\n" + - "num: 1 lower_bound: -9223372036854775808 upper_bound: 2 repeats: 0 ndv: 0\n" + - "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 0\n" + - "num: 9 lower_bound: 4 upper_bound: 12 repeats: 0 ndv: 0", - }, - //run 4th sql after table locked, hist should not changed - { - sql: "select * from t", - hist: "column:1 ndv:12 totColSize:0\n" + - "num: 1 lower_bound: -9223372036854775808 upper_bound: 2 repeats: 0 ndv: 0\n" + - "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 0\n" + - "num: 9 lower_bound: 4 upper_bound: 12 repeats: 0 ndv: 0", - }, - } - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - for i, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, err) - require.NoError(t, h.Update(is)) - tblInfo := table.Meta() - tbl := h.GetTableStats(tblInfo) - //fmt.Printf("\n i: %d, exp: %s, \nact: %s\n", i, tests[i].hist, tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) - require.Equal(t, tests[i].hist, tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) - // add table lock after 2nd - if i == 0 { - testKit.MustExec("lock stats t") - } else if i == 1 { - testKit.MustExec("unlock stats t") - } else if i == 2 { - testKit.MustExec("lock stats t") - } - } -} - func TestStatsLockForDelta(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) testKit := testkit.NewTestKit(t, store) diff --git a/statistics/histogram.go b/statistics/histogram.go index 2b757d087f784..d690bfd555f34 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -237,7 +237,7 @@ const ( Version2 = 2 ) -// AnalyzeFlag is set when the statistics comes from analyze and has not been modified by feedback. +// AnalyzeFlag is set when the statistics comes from analyze. const AnalyzeFlag = 1 // IsAnalyzed checks whether this flag contains AnalyzeFlag. @@ -245,11 +245,6 @@ func IsAnalyzed(flag int64) bool { return (flag & AnalyzeFlag) > 0 } -// ResetAnalyzeFlag resets the AnalyzeFlag because it has been modified by feedback. -func ResetAnalyzeFlag(flag int64) int64 { - return flag &^ AnalyzeFlag -} - // ValueToString converts a possible encoded value to a formatted string. If the value is encoded, then // idxCols equals to number of origin values, else idxCols is 0. func ValueToString(vars *variable.SessionVars, value *types.Datum, idxCols int, idxColumnTypes []byte) (string, error) { diff --git a/types/datum.go b/types/datum.go index d03aa62464fda..8d08f73d9798b 100644 --- a/types/datum.go +++ b/types/datum.go @@ -2271,7 +2271,6 @@ func (ds *datumsSorter) Len() int { } func (ds *datumsSorter) Less(i, j int) bool { - // TODO: set collation explicitly when rewrites feedback. cmp, err := ds.datums[i].Compare(ds.sc, &ds.datums[j], collate.GetCollator(ds.datums[i].Collation())) if err != nil { ds.err = errors.Trace(err) diff --git a/util/mock/context.go b/util/mock/context.go index f2dab9d9e0faa..d4e489d17ab26 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -331,9 +331,6 @@ func (c *Context) GoCtx() context.Context { return c.ctx } -// StoreQueryFeedback stores the query feedback. -func (*Context) StoreQueryFeedback(_ interface{}) {} - // UpdateColStatsUsage updates the column stats usage. func (*Context) UpdateColStatsUsage(_ []model.TableItemID) {} From 0c3bbc127c8e1c5754ec53ce6cbf3542fbaf8647 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 21 Aug 2023 20:36:34 +0800 Subject: [PATCH 27/32] planner: fix the issue that TIDB_INLJ hint cannot take effect when left joining two sub-queries (#46271) close pingcap/tidb#46160 --- executor/index_lookup_join_test.go | 10 ++++++++++ planner/core/logical_plan_builder.go | 3 ++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go index 6b5d6e96bcc44..b3910fdd1a872 100644 --- a/executor/index_lookup_join_test.go +++ b/executor/index_lookup_join_test.go @@ -183,6 +183,16 @@ func TestInapplicableIndexJoinHint(t *testing.T) { tk.MustQuery(`show warnings;`).Check(testkit.Rows(`Warning 1815 Optimizer Hint /*+ INL_MERGE_JOIN(t1) */ is inapplicable`)) tk.MustQuery(`select /*+ INL_MERGE_JOIN(t2) */ * from t1 right join t2 on t1.a=t2.a;`).Check(testkit.Rows()) tk.MustQuery(`show warnings;`).Check(testkit.Rows(`Warning 1815 Optimizer Hint /*+ INL_MERGE_JOIN(t2) */ is inapplicable`)) + + // Test for issues/46160 + tk.MustExec(`drop table if exists t1, t2;`) + tk.MustExec("use test") + tk.MustExec(`create table t1 (a int, key(a))`) + tk.MustExec(`create table t2 (a int, key(a))`) + + query := `select /*+ tidb_inlj(bb) */ aa.* from (select * from t1) as aa left join + (select t2.a, t2.a*2 as a2 from t2) as bb on aa.a=bb.a;` + tk.HasPlan(query, "IndexJoin") } func TestIndexJoinOverflow(t *testing.T) { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index b0216c1b7f320..86bc02b47ce38 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -714,7 +714,8 @@ func extractTableAlias(p Plan, parentOffset int) *hintTableInfo { if len(p.OutputNames()) > 0 && p.OutputNames()[0].TblName.L != "" { firstName := p.OutputNames()[0] for _, name := range p.OutputNames() { - if name.TblName.L != firstName.TblName.L || name.DBName.L != firstName.DBName.L { + if name.TblName.L != firstName.TblName.L || + (name.DBName.L != "" && firstName.DBName.L != "" && name.DBName.L != firstName.DBName.L) { // DBName can be nil, see #46160 return nil } } From ec60369299983d7677c1fa2544d71e5f54335e7d Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Tue, 22 Aug 2023 11:15:33 +0800 Subject: [PATCH 28/32] import into: split real-tikv test (#46299) ref pingcap/tidb#42930 --- Makefile | 21 +++++ .../realtikvtest/importintotest2/BUILD.bazel | 22 +++++ .../importintotest2/dummy_test.go | 19 ++++ .../realtikvtest/importintotest2/main_test.go | 93 +++++++++++++++++++ .../realtikvtest/importintotest3/BUILD.bazel | 22 +++++ .../importintotest3/dummy_test.go | 19 ++++ .../realtikvtest/importintotest3/main_test.go | 93 +++++++++++++++++++ .../realtikvtest/importintotest4/BUILD.bazel | 22 +++++ .../importintotest4/dummy_test.go | 19 ++++ .../realtikvtest/importintotest4/main_test.go | 93 +++++++++++++++++++ 10 files changed, 423 insertions(+) create mode 100644 tests/realtikvtest/importintotest2/BUILD.bazel create mode 100644 tests/realtikvtest/importintotest2/dummy_test.go create mode 100644 tests/realtikvtest/importintotest2/main_test.go create mode 100644 tests/realtikvtest/importintotest3/BUILD.bazel create mode 100644 tests/realtikvtest/importintotest3/dummy_test.go create mode 100644 tests/realtikvtest/importintotest3/main_test.go create mode 100644 tests/realtikvtest/importintotest4/BUILD.bazel create mode 100644 tests/realtikvtest/importintotest4/dummy_test.go create mode 100644 tests/realtikvtest/importintotest4/main_test.go diff --git a/Makefile b/Makefile index 82959c269663c..81348bfa3c8cb 100644 --- a/Makefile +++ b/Makefile @@ -564,6 +564,27 @@ bazel_importintotest: failpoint-enable bazel_ci_simple_prepare -- //tests/realtikvtest/importintotest/... ./build/jenkins_collect_coverage.sh +# on timeout, bazel won't print log sometimes, so we use --test_output=all to print log always +bazel_importintotest2: failpoint-enable bazel_ci_simple_prepare + bazel $(BAZEL_GLOBAL_CONFIG) coverage $(BAZEL_CMD_CONFIG) --test_output=all --test_arg=-with-real-tikv --define gotags=deadlock,intest \ + --@io_bazel_rules_go//go/config:cover_format=go_cover \ + -- //tests/realtikvtest/importintotest2/... + ./build/jenkins_collect_coverage.sh + +# on timeout, bazel won't print log sometimes, so we use --test_output=all to print log always +bazel_importintotest3: failpoint-enable bazel_ci_simple_prepare + bazel $(BAZEL_GLOBAL_CONFIG) coverage $(BAZEL_CMD_CONFIG) --test_output=all --test_arg=-with-real-tikv --define gotags=deadlock,intest \ + --@io_bazel_rules_go//go/config:cover_format=go_cover \ + -- //tests/realtikvtest/importintotest3/... + ./build/jenkins_collect_coverage.sh + +# on timeout, bazel won't print log sometimes, so we use --test_output=all to print log always +bazel_importintotest4: failpoint-enable bazel_ci_simple_prepare + bazel $(BAZEL_GLOBAL_CONFIG) coverage $(BAZEL_CMD_CONFIG) --test_output=all --test_arg=-with-real-tikv --define gotags=deadlock,intest \ + --@io_bazel_rules_go//go/config:cover_format=go_cover \ + -- //tests/realtikvtest/importintotest4/... + ./build/jenkins_collect_coverage.sh + bazel_lint: bazel_prepare bazel build //... --//build:with_nogo_flag=true diff --git a/tests/realtikvtest/importintotest2/BUILD.bazel b/tests/realtikvtest/importintotest2/BUILD.bazel new file mode 100644 index 0000000000000..000264e402a9d --- /dev/null +++ b/tests/realtikvtest/importintotest2/BUILD.bazel @@ -0,0 +1,22 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "importintotest2_test", + timeout = "moderate", + srcs = [ + "dummy_test.go", + "main_test.go", + ], + flaky = True, + race = "on", + deps = [ + "//config", + "//kv", + "//testkit", + "//tests/realtikvtest", + "@com_github_fsouza_fake_gcs_server//fakestorage", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_stretchr_testify//require", + "@com_github_stretchr_testify//suite", + ], +) diff --git a/tests/realtikvtest/importintotest2/dummy_test.go b/tests/realtikvtest/importintotest2/dummy_test.go new file mode 100644 index 0000000000000..de58a6c9a8323 --- /dev/null +++ b/tests/realtikvtest/importintotest2/dummy_test.go @@ -0,0 +1,19 @@ +// 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 importintotest + +func (s *mockGCSSuite) TestDummy() { + s.True(true, gcsEndpoint) +} diff --git a/tests/realtikvtest/importintotest2/main_test.go b/tests/realtikvtest/importintotest2/main_test.go new file mode 100644 index 0000000000000..07c466b298680 --- /dev/null +++ b/tests/realtikvtest/importintotest2/main_test.go @@ -0,0 +1,93 @@ +// 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 importintotest + +import ( + "fmt" + "testing" + + "github.com/fsouza/fake-gcs-server/fakestorage" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/tests/realtikvtest" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" +) + +type mockGCSSuite struct { + suite.Suite + + server *fakestorage.Server + store kv.Storage + tk *testkit.TestKit +} + +var ( + gcsHost = "127.0.0.1" + gcsPort = uint16(4443) + // for fake gcs server, we must use this endpoint format + // NOTE: must end with '/' + gcsEndpointFormat = "http://%s:%d/storage/v1/" + gcsEndpoint = fmt.Sprintf(gcsEndpointFormat, gcsHost, gcsPort) +) + +func TestLoadRemote(t *testing.T) { + suite.Run(t, &mockGCSSuite{}) +} + +func (s *mockGCSSuite) SetupSuite() { + s.Require().True(*realtikvtest.WithRealTiKV) + var err error + opt := fakestorage.Options{ + Scheme: "http", + Host: gcsHost, + Port: gcsPort, + PublicHost: gcsHost, + } + s.server, err = fakestorage.NewServerWithOptions(opt) + s.Require().NoError(err) + s.store = realtikvtest.CreateMockStoreAndSetup(s.T()) + s.tk = testkit.NewTestKit(s.T(), s.store) +} + +func (s *mockGCSSuite) TearDownSuite() { + s.server.Stop() +} + +func (s *mockGCSSuite) enableFailpoint(path, term string) { + require.NoError(s.T(), failpoint.Enable(path, term)) + s.T().Cleanup(func() { + _ = failpoint.Disable(path) + }) +} + +func (s *mockGCSSuite) cleanupSysTables() { + s.tk.MustExec("delete from mysql.tidb_import_jobs") + s.tk.MustExec("delete from mysql.tidb_global_task") + s.tk.MustExec("delete from mysql.tidb_background_subtask") +} + +func init() { + // need a real PD + config.UpdateGlobal(func(conf *config.Config) { + conf.Path = "127.0.0.1:2379" + }) +} + +func TestMain(m *testing.M) { + realtikvtest.RunTestMain(m) +} diff --git a/tests/realtikvtest/importintotest3/BUILD.bazel b/tests/realtikvtest/importintotest3/BUILD.bazel new file mode 100644 index 0000000000000..c7f783d02987e --- /dev/null +++ b/tests/realtikvtest/importintotest3/BUILD.bazel @@ -0,0 +1,22 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "importintotest3_test", + timeout = "moderate", + srcs = [ + "dummy_test.go", + "main_test.go", + ], + flaky = True, + race = "on", + deps = [ + "//config", + "//kv", + "//testkit", + "//tests/realtikvtest", + "@com_github_fsouza_fake_gcs_server//fakestorage", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_stretchr_testify//require", + "@com_github_stretchr_testify//suite", + ], +) diff --git a/tests/realtikvtest/importintotest3/dummy_test.go b/tests/realtikvtest/importintotest3/dummy_test.go new file mode 100644 index 0000000000000..de58a6c9a8323 --- /dev/null +++ b/tests/realtikvtest/importintotest3/dummy_test.go @@ -0,0 +1,19 @@ +// 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 importintotest + +func (s *mockGCSSuite) TestDummy() { + s.True(true, gcsEndpoint) +} diff --git a/tests/realtikvtest/importintotest3/main_test.go b/tests/realtikvtest/importintotest3/main_test.go new file mode 100644 index 0000000000000..07c466b298680 --- /dev/null +++ b/tests/realtikvtest/importintotest3/main_test.go @@ -0,0 +1,93 @@ +// 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 importintotest + +import ( + "fmt" + "testing" + + "github.com/fsouza/fake-gcs-server/fakestorage" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/tests/realtikvtest" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" +) + +type mockGCSSuite struct { + suite.Suite + + server *fakestorage.Server + store kv.Storage + tk *testkit.TestKit +} + +var ( + gcsHost = "127.0.0.1" + gcsPort = uint16(4443) + // for fake gcs server, we must use this endpoint format + // NOTE: must end with '/' + gcsEndpointFormat = "http://%s:%d/storage/v1/" + gcsEndpoint = fmt.Sprintf(gcsEndpointFormat, gcsHost, gcsPort) +) + +func TestLoadRemote(t *testing.T) { + suite.Run(t, &mockGCSSuite{}) +} + +func (s *mockGCSSuite) SetupSuite() { + s.Require().True(*realtikvtest.WithRealTiKV) + var err error + opt := fakestorage.Options{ + Scheme: "http", + Host: gcsHost, + Port: gcsPort, + PublicHost: gcsHost, + } + s.server, err = fakestorage.NewServerWithOptions(opt) + s.Require().NoError(err) + s.store = realtikvtest.CreateMockStoreAndSetup(s.T()) + s.tk = testkit.NewTestKit(s.T(), s.store) +} + +func (s *mockGCSSuite) TearDownSuite() { + s.server.Stop() +} + +func (s *mockGCSSuite) enableFailpoint(path, term string) { + require.NoError(s.T(), failpoint.Enable(path, term)) + s.T().Cleanup(func() { + _ = failpoint.Disable(path) + }) +} + +func (s *mockGCSSuite) cleanupSysTables() { + s.tk.MustExec("delete from mysql.tidb_import_jobs") + s.tk.MustExec("delete from mysql.tidb_global_task") + s.tk.MustExec("delete from mysql.tidb_background_subtask") +} + +func init() { + // need a real PD + config.UpdateGlobal(func(conf *config.Config) { + conf.Path = "127.0.0.1:2379" + }) +} + +func TestMain(m *testing.M) { + realtikvtest.RunTestMain(m) +} diff --git a/tests/realtikvtest/importintotest4/BUILD.bazel b/tests/realtikvtest/importintotest4/BUILD.bazel new file mode 100644 index 0000000000000..4a8595bf816eb --- /dev/null +++ b/tests/realtikvtest/importintotest4/BUILD.bazel @@ -0,0 +1,22 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "importintotest4_test", + timeout = "moderate", + srcs = [ + "dummy_test.go", + "main_test.go", + ], + flaky = True, + race = "on", + deps = [ + "//config", + "//kv", + "//testkit", + "//tests/realtikvtest", + "@com_github_fsouza_fake_gcs_server//fakestorage", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_stretchr_testify//require", + "@com_github_stretchr_testify//suite", + ], +) diff --git a/tests/realtikvtest/importintotest4/dummy_test.go b/tests/realtikvtest/importintotest4/dummy_test.go new file mode 100644 index 0000000000000..de58a6c9a8323 --- /dev/null +++ b/tests/realtikvtest/importintotest4/dummy_test.go @@ -0,0 +1,19 @@ +// 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 importintotest + +func (s *mockGCSSuite) TestDummy() { + s.True(true, gcsEndpoint) +} diff --git a/tests/realtikvtest/importintotest4/main_test.go b/tests/realtikvtest/importintotest4/main_test.go new file mode 100644 index 0000000000000..07c466b298680 --- /dev/null +++ b/tests/realtikvtest/importintotest4/main_test.go @@ -0,0 +1,93 @@ +// 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 importintotest + +import ( + "fmt" + "testing" + + "github.com/fsouza/fake-gcs-server/fakestorage" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/tests/realtikvtest" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" +) + +type mockGCSSuite struct { + suite.Suite + + server *fakestorage.Server + store kv.Storage + tk *testkit.TestKit +} + +var ( + gcsHost = "127.0.0.1" + gcsPort = uint16(4443) + // for fake gcs server, we must use this endpoint format + // NOTE: must end with '/' + gcsEndpointFormat = "http://%s:%d/storage/v1/" + gcsEndpoint = fmt.Sprintf(gcsEndpointFormat, gcsHost, gcsPort) +) + +func TestLoadRemote(t *testing.T) { + suite.Run(t, &mockGCSSuite{}) +} + +func (s *mockGCSSuite) SetupSuite() { + s.Require().True(*realtikvtest.WithRealTiKV) + var err error + opt := fakestorage.Options{ + Scheme: "http", + Host: gcsHost, + Port: gcsPort, + PublicHost: gcsHost, + } + s.server, err = fakestorage.NewServerWithOptions(opt) + s.Require().NoError(err) + s.store = realtikvtest.CreateMockStoreAndSetup(s.T()) + s.tk = testkit.NewTestKit(s.T(), s.store) +} + +func (s *mockGCSSuite) TearDownSuite() { + s.server.Stop() +} + +func (s *mockGCSSuite) enableFailpoint(path, term string) { + require.NoError(s.T(), failpoint.Enable(path, term)) + s.T().Cleanup(func() { + _ = failpoint.Disable(path) + }) +} + +func (s *mockGCSSuite) cleanupSysTables() { + s.tk.MustExec("delete from mysql.tidb_import_jobs") + s.tk.MustExec("delete from mysql.tidb_global_task") + s.tk.MustExec("delete from mysql.tidb_background_subtask") +} + +func init() { + // need a real PD + config.UpdateGlobal(func(conf *config.Config) { + conf.Path = "127.0.0.1:2379" + }) +} + +func TestMain(m *testing.M) { + realtikvtest.RunTestMain(m) +} From 7e476a502759530e42130d9074230c4f79a15811 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Tue, 22 Aug 2023 11:56:04 +0800 Subject: [PATCH 29/32] *: use std/slices to replace exp/slices (#46292) ref pingcap/tidb#45933 --- br/pkg/lightning/backend/external/BUILD.bazel | 1 - .../lightning/backend/external/writer_test.go | 6 ++-- br/pkg/lightning/backend/local/BUILD.bazel | 1 - br/pkg/lightning/backend/local/disk_quota.go | 13 +++++--- br/pkg/lightning/importer/BUILD.bazel | 1 - br/pkg/lightning/importer/dup_detect_test.go | 2 +- ddl/ddl.go | 3 +- executor/executor.go | 11 ++++--- executor/index_lookup_merge_join.go | 24 ++++++++------ executor/sort.go | 31 ++++++++++++++++--- planner/core/BUILD.bazel | 1 - planner/core/optimizer.go | 23 +++++++------- planner/funcdep/BUILD.bazel | 1 - planner/funcdep/fast_int_set_test.go | 2 +- sessionctx/stmtctx/BUILD.bazel | 1 - sessionctx/stmtctx/stmtctx.go | 2 +- statistics/BUILD.bazel | 1 - statistics/cmsketch.go | 21 ++++++++----- statistics/merge_worker.go | 2 +- statistics/selectivity.go | 9 +++--- .../mockstore/unistore/cophandler/BUILD.bazel | 1 - .../mockstore/unistore/cophandler/analyze.go | 20 ++---------- store/pdtypes/BUILD.bazel | 1 - store/pdtypes/region_tree.go | 6 ++-- table/temptable/BUILD.bazel | 1 - table/temptable/main_test.go | 4 +-- 26 files changed, 102 insertions(+), 87 deletions(-) diff --git a/br/pkg/lightning/backend/external/BUILD.bazel b/br/pkg/lightning/backend/external/BUILD.bazel index 1bba1a5b87da7..fe3cfa189d067 100644 --- a/br/pkg/lightning/backend/external/BUILD.bazel +++ b/br/pkg/lightning/backend/external/BUILD.bazel @@ -55,7 +55,6 @@ go_test( "@com_github_pingcap_errors//:errors", "@com_github_stretchr_testify//require", "@org_golang_x_exp//rand", - "@org_golang_x_exp//slices", "@org_uber_go_atomic//:atomic", ], ) diff --git a/br/pkg/lightning/backend/external/writer_test.go b/br/pkg/lightning/backend/external/writer_test.go index d12afc3562ff9..3b3abce281190 100644 --- a/br/pkg/lightning/backend/external/writer_test.go +++ b/br/pkg/lightning/backend/external/writer_test.go @@ -19,6 +19,7 @@ import ( "context" "fmt" "io" + "slices" "strings" "testing" "time" @@ -28,7 +29,6 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" "github.com/stretchr/testify/require" "golang.org/x/exp/rand" - "golang.org/x/exp/slices" ) func TestWriter(t *testing.T) { @@ -62,8 +62,8 @@ func TestWriter(t *testing.T) { _, err = writer.Close(ctx) require.NoError(t, err) - slices.SortFunc(kvs, func(i, j common.KvPair) bool { - return bytes.Compare(i.Key, j.Key) < 0 + slices.SortFunc(kvs, func(i, j common.KvPair) int { + return bytes.Compare(i.Key, j.Key) }) bufSize := rand.Intn(100) + 1 diff --git a/br/pkg/lightning/backend/local/BUILD.bazel b/br/pkg/lightning/backend/local/BUILD.bazel index 94867d6c7fd4f..feb9c00b7c2a8 100644 --- a/br/pkg/lightning/backend/local/BUILD.bazel +++ b/br/pkg/lightning/backend/local/BUILD.bazel @@ -86,7 +86,6 @@ go_library( "@org_golang_google_grpc//credentials/insecure", "@org_golang_google_grpc//keepalive", "@org_golang_google_grpc//status", - "@org_golang_x_exp//slices", "@org_golang_x_sync//errgroup", "@org_golang_x_time//rate", "@org_uber_go_atomic//:atomic", diff --git a/br/pkg/lightning/backend/local/disk_quota.go b/br/pkg/lightning/backend/local/disk_quota.go index 9f224665b7661..cc2a2f4708914 100644 --- a/br/pkg/lightning/backend/local/disk_quota.go +++ b/br/pkg/lightning/backend/local/disk_quota.go @@ -15,9 +15,11 @@ package local import ( + "cmp" + "slices" + "github.com/google/uuid" "github.com/pingcap/tidb/br/pkg/lightning/backend" - "golang.org/x/exp/slices" ) // DiskUsage is an interface to obtain the size occupied locally of all engines @@ -38,11 +40,14 @@ func CheckDiskQuota(mgr DiskUsage, quota int64) ( totalMemSize int64, ) { sizes := mgr.EngineFileSizes() - slices.SortFunc(sizes, func(i, j backend.EngineFileSize) bool { + slices.SortFunc(sizes, func(i, j backend.EngineFileSize) int { if i.IsImporting != j.IsImporting { - return i.IsImporting + if i.IsImporting { + return -1 + } + return 1 } - return i.DiskSize+i.MemSize < j.DiskSize+j.MemSize + return cmp.Compare(i.DiskSize+i.MemSize, j.DiskSize+j.MemSize) }) for _, size := range sizes { totalDiskSize += size.DiskSize diff --git a/br/pkg/lightning/importer/BUILD.bazel b/br/pkg/lightning/importer/BUILD.bazel index e52caf6f7aeee..7c8e2bc95b7a1 100644 --- a/br/pkg/lightning/importer/BUILD.bazel +++ b/br/pkg/lightning/importer/BUILD.bazel @@ -182,7 +182,6 @@ go_test( "@com_github_xitongsys_parquet_go_source//buffer", "@io_etcd_go_etcd_client_v3//:client", "@io_etcd_go_etcd_tests_v3//integration", - "@org_golang_x_exp//slices", "@org_uber_go_zap//:zap", ], ) diff --git a/br/pkg/lightning/importer/dup_detect_test.go b/br/pkg/lightning/importer/dup_detect_test.go index 2f6aaad85ea45..53f209023acc8 100644 --- a/br/pkg/lightning/importer/dup_detect_test.go +++ b/br/pkg/lightning/importer/dup_detect_test.go @@ -16,6 +16,7 @@ package importer import ( "context" + "slices" "testing" "github.com/pingcap/errors" @@ -27,7 +28,6 @@ import ( "github.com/pingcap/tidb/util/dbutil" "github.com/pingcap/tidb/util/extsort" "github.com/stretchr/testify/require" - "golang.org/x/exp/slices" ) var ( diff --git a/ddl/ddl.go b/ddl/ddl.go index 52853e10488b2..9a00e55e0afa7 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -64,7 +64,6 @@ import ( "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/syncutil" "github.com/tikv/client-go/v2/tikvrpc" clientv3 "go.etcd.io/etcd/client/v3" @@ -747,7 +746,7 @@ func (d *ddl) prepareWorkers4ConcurrencyDDL() { } } // reorg worker count at least 1 at most 10. - reorgCnt := mathutil.Min(mathutil.Max(runtime.GOMAXPROCS(0)/4, 1), reorgWorkerCnt) + reorgCnt := min(max(runtime.GOMAXPROCS(0)/4, 1), reorgWorkerCnt) d.reorgWorkerPool = newDDLWorkerPool(pools.NewResourcePool(workerFactory(addIdxWorker), reorgCnt, reorgCnt, 0), reorg) d.generalDDLWorkerPool = newDDLWorkerPool(pools.NewResourcePool(workerFactory(generalWorker), generalWorkerCnt, generalWorkerCnt, 0), general) failpoint.Inject("NoDDLDispatchLoop", func(val failpoint.Value) { diff --git a/executor/executor.go b/executor/executor.go index f7eb3dabb1a43..5749f74a2e550 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -15,10 +15,12 @@ package executor import ( + "cmp" "context" "fmt" "math" "runtime/pprof" + "slices" "strconv" "strings" "sync" @@ -82,7 +84,6 @@ import ( tikvutil "github.com/tikv/client-go/v2/util" atomicutil "go.uber.org/atomic" "go.uber.org/zap" - "golang.org/x/exp/slices" ) var ( @@ -2475,8 +2476,8 @@ func (w *checkIndexWorker) HandleTask(task checkIndexTask) (_ workerpool.None) { trySaveErr(err) return } - slices.SortFunc(tableChecksum, func(i, j groupByChecksum) bool { - return i.bucket < j.bucket + slices.SortFunc(tableChecksum, func(i, j groupByChecksum) int { + return cmp.Compare(i.bucket, j.bucket) }) // compute index side checksum. @@ -2485,8 +2486,8 @@ func (w *checkIndexWorker) HandleTask(task checkIndexTask) (_ workerpool.None) { trySaveErr(err) return } - slices.SortFunc(indexChecksum, func(i, j groupByChecksum) bool { - return i.bucket < j.bucket + slices.SortFunc(indexChecksum, func(i, j groupByChecksum) int { + return cmp.Compare(i.bucket, j.bucket) }) currentOffset := 0 diff --git a/executor/index_lookup_merge_join.go b/executor/index_lookup_merge_join.go index 64a4abab1b7d2..b7de825f0e129 100644 --- a/executor/index_lookup_merge_join.go +++ b/executor/index_lookup_merge_join.go @@ -18,6 +18,7 @@ import ( "context" "fmt" "runtime/trace" + "slices" "sync" "sync/atomic" @@ -38,7 +39,6 @@ import ( "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/ranger" "go.uber.org/zap" - "golang.org/x/exp/slices" ) // IndexLookUpMergeJoin realizes IndexLookUpJoin by merge join @@ -450,23 +450,29 @@ func (imw *innerMergeWorker) handleTask(ctx context.Context, task *lookUpMergeJo // Because the necessary condition of merge join is both outer and inner keep order of join keys. // In this case, we need sort the outer side. if imw.outerMergeCtx.needOuterSort { - slices.SortFunc(task.outerOrderIdx, func(idxI, idxJ chunk.RowPtr) bool { + slices.SortFunc(task.outerOrderIdx, func(idxI, idxJ chunk.RowPtr) int { rowI, rowJ := task.outerResult.GetRow(idxI), task.outerResult.GetRow(idxJ) - var cmp int64 + var c int64 var err error for _, keyOff := range imw.keyOff2KeyOffOrderByIdx { joinKey := imw.outerMergeCtx.joinKeys[keyOff] - cmp, _, err = imw.outerMergeCtx.compareFuncs[keyOff](imw.ctx, joinKey, joinKey, rowI, rowJ) + c, _, err = imw.outerMergeCtx.compareFuncs[keyOff](imw.ctx, joinKey, joinKey, rowI, rowJ) terror.Log(err) - if cmp != 0 { + if c != 0 { break } } - if cmp != 0 || imw.nextColCompareFilters == nil { - return (cmp < 0 && !imw.desc) || (cmp > 0 && imw.desc) + if c != 0 || imw.nextColCompareFilters == nil { + if imw.desc { + return int(-c) + } + return int(c) + } + c = int64(imw.nextColCompareFilters.CompareRow(rowI, rowJ)) + if imw.desc { + return int(-c) } - cmp = int64(imw.nextColCompareFilters.CompareRow(rowI, rowJ)) - return (cmp < 0 && !imw.desc) || (cmp > 0 && imw.desc) + return int(c) }) } dLookUpKeys, err := imw.constructDatumLookupKeys(task) diff --git a/executor/sort.go b/executor/sort.go index 4210a086bcf30..5ccdab75d3db5 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -18,6 +18,7 @@ import ( "container/heap" "context" "errors" + "slices" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/executor/internal/exec" @@ -29,7 +30,6 @@ import ( "github.com/pingcap/tidb/util/disk" "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/memory" - "golang.org/x/exp/slices" ) // SortExec represents sorting executor. @@ -143,7 +143,7 @@ func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { if e.multiWayMerge == nil { - e.multiWayMerge = &multiWayMerge{e.lessRow, make([]partitionPointer, 0, len(e.partitionList))} + e.multiWayMerge = &multiWayMerge{e.lessRow, e.compressRow, make([]partitionPointer, 0, len(e.partitionList))} for i := 0; i < len(e.partitionList); i++ { row, err := e.partitionList[i].GetSortedRow(0) if err != nil { @@ -273,6 +273,20 @@ func (e *SortExec) lessRow(rowI, rowJ chunk.Row) bool { return false } +func (e *SortExec) compressRow(rowI, rowJ chunk.Row) int { + for i, colIdx := range e.keyColumns { + cmpFunc := e.keyCmpFuncs[i] + cmp := cmpFunc(rowI, colIdx, rowJ, colIdx) + if e.ByItems[i].Desc { + cmp = -cmp + } + if cmp != 0 { + return cmp + } + } + return 0 +} + type partitionPointer struct { row chunk.Row partitionID int @@ -280,8 +294,9 @@ type partitionPointer struct { } type multiWayMerge struct { - lessRowFunction func(rowI chunk.Row, rowJ chunk.Row) bool - elements []partitionPointer + lessRowFunction func(rowI chunk.Row, rowJ chunk.Row) bool + compressRowFunction func(rowI chunk.Row, rowJ chunk.Row) int + elements []partitionPointer } func (h *multiWayMerge) Less(i, j int) bool { @@ -376,6 +391,12 @@ func (e *TopNExec) keyColumnsLess(i, j chunk.RowPtr) bool { return e.lessRow(rowI, rowJ) } +func (e *TopNExec) keyColumnsCompare(i, j chunk.RowPtr) int { + rowI := e.rowChunks.GetRow(i) + rowJ := e.rowChunks.GetRow(j) + return e.compressRow(rowI, rowJ) +} + func (e *TopNExec) initPointers() { e.rowPtrs = make([]chunk.RowPtr, 0, e.rowChunks.Len()) e.memTracker.Consume(int64(8 * e.rowChunks.Len())) @@ -481,7 +502,7 @@ func (e *TopNExec) executeTopN(ctx context.Context) error { } } } - slices.SortFunc(e.rowPtrs, e.keyColumnsLess) + slices.SortFunc(e.rowPtrs, e.keyColumnsCompare) return nil } diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index d18fb0424f71a..c17af4b188ea6 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -171,7 +171,6 @@ go_library( "@com_github_tikv_client_go_v2//kv", "@com_github_tikv_client_go_v2//oracle", "@com_github_tikv_client_go_v2//tikv", - "@org_golang_x_exp//slices", "@org_uber_go_atomic//:atomic", "@org_uber_go_zap//:zap", ], diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index bf2d055c9b4bc..7519935dfe8eb 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -15,10 +15,12 @@ package core import ( + "cmp" "context" "fmt" "math" "runtime" + "slices" "strconv" "time" @@ -50,7 +52,6 @@ import ( "github.com/pingcap/tipb/go-tipb" "go.uber.org/atomic" "go.uber.org/zap" - "golang.org/x/exp/slices" ) // OptimizeAstNode optimizes the query to a physical plan directly. @@ -345,24 +346,24 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic func refineCETrace(sctx sessionctx.Context) { stmtCtx := sctx.GetSessionVars().StmtCtx stmtCtx.OptimizerCETrace = tracing.DedupCETrace(stmtCtx.OptimizerCETrace) - slices.SortFunc(stmtCtx.OptimizerCETrace, func(i, j *tracing.CETraceRecord) bool { + slices.SortFunc(stmtCtx.OptimizerCETrace, func(i, j *tracing.CETraceRecord) int { if i == nil && j != nil { - return true + return -1 } if i == nil || j == nil { - return false + return 1 } - if i.TableID != j.TableID { - return i.TableID < j.TableID + if c := cmp.Compare(i.TableID, j.TableID); c != 0 { + return c } - if i.Type != j.Type { - return i.Type < j.Type + if c := cmp.Compare(i.Type, j.Type); c != 0 { + return c } - if i.Expr != j.Expr { - return i.Expr < j.Expr + if c := cmp.Compare(i.Expr, j.Expr); c != 0 { + return c } - return i.RowCount < j.RowCount + return cmp.Compare(i.RowCount, j.RowCount) }) traceRecords := stmtCtx.OptimizerCETrace is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema) diff --git a/planner/funcdep/BUILD.bazel b/planner/funcdep/BUILD.bazel index e5f8591c1b675..4a140ec179d78 100644 --- a/planner/funcdep/BUILD.bazel +++ b/planner/funcdep/BUILD.bazel @@ -41,7 +41,6 @@ go_test( "//util/hint", "@com_github_stretchr_testify//require", "@org_golang_x_exp//maps", - "@org_golang_x_exp//slices", "@org_golang_x_tools//container/intsets", "@org_uber_go_goleak//:goleak", ], diff --git a/planner/funcdep/fast_int_set_test.go b/planner/funcdep/fast_int_set_test.go index 72cfef58ec870..110484f2849a2 100644 --- a/planner/funcdep/fast_int_set_test.go +++ b/planner/funcdep/fast_int_set_test.go @@ -19,13 +19,13 @@ import ( "math/rand" "reflect" "runtime" + "slices" "strings" "testing" "time" "github.com/stretchr/testify/require" "golang.org/x/exp/maps" - "golang.org/x/exp/slices" "golang.org/x/tools/container/intsets" ) diff --git a/sessionctx/stmtctx/BUILD.bazel b/sessionctx/stmtctx/BUILD.bazel index e16d9a44da20d..282d6332225a8 100644 --- a/sessionctx/stmtctx/BUILD.bazel +++ b/sessionctx/stmtctx/BUILD.bazel @@ -23,7 +23,6 @@ go_library( "@com_github_tikv_client_go_v2//tikvrpc", "@com_github_tikv_client_go_v2//util", "@org_golang_x_exp//maps", - "@org_golang_x_exp//slices", "@org_uber_go_atomic//:atomic", "@org_uber_go_zap//:zap", ], diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 5e3ead54d0640..676647635869e 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -20,6 +20,7 @@ import ( "fmt" "io" "math" + "slices" "strconv" "strings" "sync" @@ -45,7 +46,6 @@ import ( atomic2 "go.uber.org/atomic" "go.uber.org/zap" "golang.org/x/exp/maps" - "golang.org/x/exp/slices" ) const ( diff --git a/statistics/BUILD.bazel b/statistics/BUILD.bazel index ff7c8f7cae493..9fedac82974df 100644 --- a/statistics/BUILD.bazel +++ b/statistics/BUILD.bazel @@ -58,7 +58,6 @@ go_library( "@com_github_pingcap_tipb//go-tipb", "@com_github_twmb_murmur3//:murmur3", "@org_golang_x_exp//maps", - "@org_golang_x_exp//slices", "@org_uber_go_atomic//:atomic", "@org_uber_go_zap//:zap", ], diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index f94d00ca08fa5..6c49861a77723 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -118,7 +118,7 @@ func newTopNHelper(sample [][]byte, numTop uint32) *topNHelper { sumTopN uint64 sampleNDV = uint32(len(sorted)) ) - numTop = mathutil.Min(sampleNDV, numTop) // Ensure numTop no larger than sampNDV. + numTop = min(sampleNDV, numTop) // Ensure numTop no larger than sampNDV. // Only element whose frequency is not smaller than 2/3 multiples the // frequency of the n-th element are added to the TopN statistics. We chose // 2/3 as an empirical value because the average cardinality estimation @@ -876,7 +876,7 @@ func MergePartTopN2GlobalTopN(loc *time.Location, version int, topNs []*TopN, n data := hack.Slice(string(value)) sorted = append(sorted, TopNMeta{Encoded: data, Count: uint64(cnt)}) } - globalTopN, leftTopN := getMergedTopNFromSortedSlice(sorted, n) + globalTopN, leftTopN := GetMergedTopNFromSortedSlice(sorted, n) return globalTopN, leftTopN, hists, nil } @@ -907,7 +907,7 @@ func MergeTopN(topNs []*TopN, n uint32) (*TopN, []TopNMeta) { data := hack.Slice(string(value)) sorted = append(sorted, TopNMeta{Encoded: data, Count: cnt}) } - return getMergedTopNFromSortedSlice(sorted, n) + return GetMergedTopNFromSortedSlice(sorted, n) } func checkEmptyTopNs(topNs []*TopN) bool { @@ -928,14 +928,19 @@ func SortTopnMeta(topnMetas []TopNMeta) { }) } -// GetMergedTopNFromSortedSlice returns merged topn -func GetMergedTopNFromSortedSlice(sorted []TopNMeta, n uint32) (*TopN, []TopNMeta) { - return getMergedTopNFromSortedSlice(sorted, n) +// TopnMetaCompare compare topnMeta +func TopnMetaCompare(i, j TopNMeta) int { + c := cmp.Compare(i.Count, j.Count) + if c == 0 { + return c + } + return bytes.Compare(i.Encoded, j.Encoded) } -func getMergedTopNFromSortedSlice(sorted []TopNMeta, n uint32) (*TopN, []TopNMeta) { +// GetMergedTopNFromSortedSlice returns merged topn +func GetMergedTopNFromSortedSlice(sorted []TopNMeta, n uint32) (*TopN, []TopNMeta) { SortTopnMeta(sorted) - n = mathutil.Min(uint32(len(sorted)), n) + n = min(uint32(len(sorted)), n) var finalTopN TopN finalTopN.TopN = sorted[:n] diff --git a/statistics/merge_worker.go b/statistics/merge_worker.go index 3d2987eee3007..22f631a8f3dcc 100644 --- a/statistics/merge_worker.go +++ b/statistics/merge_worker.go @@ -188,7 +188,7 @@ func (worker *topnStatsMergeWorker) Run(timeZone *time.Location, isIndex bool, data := hack.Slice(string(value)) sorted = append(sorted, TopNMeta{Encoded: data, Count: uint64(cnt)}) } - globalTopN, leftTopN := getMergedTopNFromSortedSlice(sorted, n) + globalTopN, leftTopN := GetMergedTopNFromSortedSlice(sorted, n) resp.TopN = globalTopN resp.PopedTopn = leftTopN worker.respCh <- resp diff --git a/statistics/selectivity.go b/statistics/selectivity.go index b49da070a7ada..a5b80961c5dee 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -16,8 +16,10 @@ package statistics import ( "bytes" + "cmp" "math" "math/bits" + "slices" "github.com/pingcap/errors" "github.com/pingcap/tidb/expression" @@ -35,7 +37,6 @@ import ( "github.com/pingcap/tidb/util/tracing" "go.uber.org/zap" "golang.org/x/exp/maps" - "golang.org/x/exp/slices" ) // If one condition can't be calculated, we will assume that the selectivity of this condition is 0.8. @@ -590,11 +591,11 @@ func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, ran // GetUsableSetsByGreedy will select the indices and pk used for calculate selectivity by greedy algorithm. func GetUsableSetsByGreedy(nodes []*StatsNode) (newBlocks []*StatsNode) { - slices.SortFunc(nodes, func(i, j *StatsNode) bool { + slices.SortFunc(nodes, func(i, j *StatsNode) int { if r := compareType(i.Tp, j.Tp); r != 0 { - return r < 0 + return r } - return i.ID < j.ID + return cmp.Compare(i.ID, j.ID) }) marked := make([]bool, len(nodes)) mask := int64(math.MaxInt64) diff --git a/store/mockstore/unistore/cophandler/BUILD.bazel b/store/mockstore/unistore/cophandler/BUILD.bazel index c1e32c7bab2a8..d4dcc21200db7 100644 --- a/store/mockstore/unistore/cophandler/BUILD.bazel +++ b/store/mockstore/unistore/cophandler/BUILD.bazel @@ -48,7 +48,6 @@ go_library( "@com_github_pingcap_tipb//go-tipb", "@com_github_tikv_client_go_v2//tikvrpc", "@com_github_twmb_murmur3//:murmur3", - "@org_golang_x_exp//slices", "@org_uber_go_atomic//:atomic", ], ) diff --git a/store/mockstore/unistore/cophandler/analyze.go b/store/mockstore/unistore/cophandler/analyze.go index a04b158e4b4a6..626147ed16d22 100644 --- a/store/mockstore/unistore/cophandler/analyze.go +++ b/store/mockstore/unistore/cophandler/analyze.go @@ -19,6 +19,7 @@ import ( "context" "math" "math/rand" + "slices" "time" "github.com/golang/protobuf/proto" @@ -40,7 +41,6 @@ import ( "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tipb/go-tipb" "github.com/twmb/murmur3" - "golang.org/x/exp/slices" ) // handleCopAnalyzeRequest handles coprocessor analyze request. @@ -112,14 +112,7 @@ func handleAnalyzeIndexReq(dbReader *dbreader.DBReader, rans []kv.KeyRange, anal if processor.topNCurValuePair.Count != 0 { processor.topNValuePairs = append(processor.topNValuePairs, processor.topNCurValuePair) } - slices.SortFunc(processor.topNValuePairs, func(i, j statistics.TopNMeta) bool { - if i.Count > j.Count { - return true - } else if i.Count < j.Count { - return false - } - return bytes.Compare(i.Encoded, j.Encoded) < 0 - }) + slices.SortFunc(processor.topNValuePairs, statistics.TopnMetaCompare) if len(processor.topNValuePairs) > int(processor.topNCount) { processor.topNValuePairs = processor.topNValuePairs[:processor.topNCount] } @@ -564,14 +557,7 @@ func handleAnalyzeMixedReq(dbReader *dbreader.DBReader, rans []kv.KeyRange, anal if e.topNCurValuePair.Count != 0 { e.topNValuePairs = append(e.topNValuePairs, e.topNCurValuePair) } - slices.SortFunc(e.topNValuePairs, func(i, j statistics.TopNMeta) bool { - if i.Count > j.Count { - return true - } else if i.Count < j.Count { - return false - } - return bytes.Compare(i.Encoded, j.Encoded) < 0 - }) + slices.SortFunc(e.topNValuePairs, statistics.TopnMetaCompare) if len(e.topNValuePairs) > int(e.topNCount) { e.topNValuePairs = e.topNValuePairs[:e.topNCount] } diff --git a/store/pdtypes/BUILD.bazel b/store/pdtypes/BUILD.bazel index dd35cae23c4d9..fe59445be494a 100644 --- a/store/pdtypes/BUILD.bazel +++ b/store/pdtypes/BUILD.bazel @@ -17,6 +17,5 @@ go_library( "@com_github_pingcap_errors//:errors", "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pingcap_kvproto//pkg/pdpb", - "@org_golang_x_exp//slices", ], ) diff --git a/store/pdtypes/region_tree.go b/store/pdtypes/region_tree.go index cc99e981e0097..5112b7b810d6b 100644 --- a/store/pdtypes/region_tree.go +++ b/store/pdtypes/region_tree.go @@ -16,9 +16,9 @@ package pdtypes import ( "bytes" + "slices" "github.com/pingcap/kvproto/pkg/metapb" - "golang.org/x/exp/slices" ) // Region is a mock of PD's core.RegionInfo. For testing purpose. @@ -52,8 +52,8 @@ func (t *RegionTree) SetRegion(region *Region) { // ScanRange scans regions intersecting [start key, end key), returns at most // `limit` regions. limit <= 0 means no limit. func (t *RegionTree) ScanRange(startKey, endKey []byte, limit int) []*Region { - slices.SortFunc(t.Regions, func(i, j *Region) bool { - return bytes.Compare(i.Meta.StartKey, j.Meta.StartKey) < 0 + slices.SortFunc(t.Regions, func(i, j *Region) int { + return bytes.Compare(i.Meta.StartKey, j.Meta.StartKey) }) pivot := NewRegionInfo(&metapb.Region{StartKey: startKey, EndKey: endKey}, nil) var res []*Region diff --git a/table/temptable/BUILD.bazel b/table/temptable/BUILD.bazel index 6b37476e145f5..fa977d1b267a6 100644 --- a/table/temptable/BUILD.bazel +++ b/table/temptable/BUILD.bazel @@ -59,7 +59,6 @@ go_test( "//util/mock", "@com_github_pingcap_errors//:errors", "@com_github_stretchr_testify//require", - "@org_golang_x_exp//slices", "@org_uber_go_goleak//:goleak", ], ) diff --git a/table/temptable/main_test.go b/table/temptable/main_test.go index 8b8487b2eceba..94bc60c619666 100644 --- a/table/temptable/main_test.go +++ b/table/temptable/main_test.go @@ -18,6 +18,7 @@ import ( "bytes" "context" "fmt" + "slices" "testing" "github.com/pingcap/tidb/infoschema" @@ -31,7 +32,6 @@ import ( "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" "go.uber.org/goleak" - "golang.org/x/exp/slices" ) func TestMain(m *testing.M) { @@ -126,7 +126,7 @@ func newMockedRetriever(t *testing.T) *mockedRetriever { } func (r *mockedRetriever) SetData(data []*kv.Entry) *mockedRetriever { - lessFunc := func(i, j *kv.Entry) bool { return bytes.Compare(i.Key, j.Key) < 0 } + lessFunc := func(i, j *kv.Entry) int { return bytes.Compare(i.Key, j.Key) } if !slices.IsSortedFunc(data, lessFunc) { data = append([]*kv.Entry{}, data...) slices.SortFunc(data, lessFunc) From a8cfe88abae6ce83889d3c2e811f412656c70a40 Mon Sep 17 00:00:00 2001 From: YangKeao Date: Tue, 22 Aug 2023 15:09:34 +0800 Subject: [PATCH 30/32] util/collate: add collation utf8mb4_0900_bin (#46269) close pingcap/tidb#46268 --- .../r/collation_misc_enabled.result | 2 + executor/test/seqtest/seq_executor_test.go | 1 + util/collate/collate.go | 2 + util/collate/collate_bench_test.go | 36 ++++++++++-- util/collate/collate_test.go | 57 +++++++++++-------- 5 files changed, 67 insertions(+), 31 deletions(-) diff --git a/cmd/explaintest/r/collation_misc_enabled.result b/cmd/explaintest/r/collation_misc_enabled.result index 4d8015bc67302..4e4db999b53bd 100644 --- a/cmd/explaintest/r/collation_misc_enabled.result +++ b/cmd/explaintest/r/collation_misc_enabled.result @@ -105,6 +105,7 @@ utf8 83 1 utf8 33 1 utf8 192 1 utf8mb4 255 1 +utf8mb4 309 1 utf8mb4 46 1 utf8mb4 45 1 utf8mb4 224 1 @@ -130,6 +131,7 @@ utf8_bin utf8 83 Yes Yes 1 utf8_general_ci utf8 33 Yes 1 utf8_unicode_ci utf8 192 Yes 1 utf8mb4_0900_ai_ci utf8mb4 255 Yes 1 +utf8mb4_0900_bin utf8mb4 309 Yes 1 utf8mb4_bin utf8mb4 46 Yes Yes 1 utf8mb4_general_ci utf8mb4 45 Yes 1 utf8mb4_unicode_ci utf8mb4 224 Yes 1 diff --git a/executor/test/seqtest/seq_executor_test.go b/executor/test/seqtest/seq_executor_test.go index 1f5d34b2526e8..b18181c8fb603 100644 --- a/executor/test/seqtest/seq_executor_test.go +++ b/executor/test/seqtest/seq_executor_test.go @@ -1203,6 +1203,7 @@ func TestShowForNewCollations(t *testing.T) { "utf8_general_ci utf8 33 Yes 1", "utf8_unicode_ci utf8 192 Yes 1", "utf8mb4_0900_ai_ci utf8mb4 255 Yes 1", + "utf8mb4_0900_bin utf8mb4 309 Yes 1", "utf8mb4_bin utf8mb4 46 Yes Yes 1", "utf8mb4_general_ci utf8mb4 45 Yes 1", "utf8mb4_unicode_ci utf8mb4 224 Yes 1", diff --git a/util/collate/collate.go b/util/collate/collate.go index 05d1a4750a741..0b544b00d8883 100644 --- a/util/collate/collate.go +++ b/util/collate/collate.go @@ -404,6 +404,8 @@ func init() { newCollatorIDMap[CollationName2ID("utf8mb4_bin")] = &binPaddingCollator{} newCollatorMap["utf8_bin"] = &binPaddingCollator{} newCollatorIDMap[CollationName2ID("utf8_bin")] = &binPaddingCollator{} + newCollatorMap["utf8mb4_0900_bin"] = &binCollator{} + newCollatorIDMap[CollationName2ID("utf8mb4_0900_bin")] = &binCollator{} newCollatorMap["utf8mb4_general_ci"] = &generalCICollator{} newCollatorIDMap[CollationName2ID("utf8mb4_general_ci")] = &generalCICollator{} newCollatorMap["utf8_general_ci"] = &generalCICollator{} diff --git a/util/collate/collate_bench_test.go b/util/collate/collate_bench_test.go index cad9e254d19ae..95707c6ec4511 100644 --- a/util/collate/collate_bench_test.go +++ b/util/collate/collate_bench_test.go @@ -54,7 +54,7 @@ func key(b *testing.B, collator Collator, length int) { } func BenchmarkUtf8mb4Bin_CompareShort(b *testing.B) { - compare(b, &binCollator{}, short) + compare(b, &binPaddingCollator{}, short) } func BenchmarkUtf8mb4GeneralCI_CompareShort(b *testing.B) { @@ -69,8 +69,12 @@ func BenchmarkUtf8mb40900AICI_CompareShort(b *testing.B) { compare(b, &unicode0900AICICollator{}, short) } +func BenchmarkUtf8mb40900Bin_CompareShort(b *testing.B) { + compare(b, &binCollator{}, short) +} + func BenchmarkUtf8mb4Bin_CompareMid(b *testing.B) { - compare(b, &binCollator{}, middle) + compare(b, &binPaddingCollator{}, middle) } func BenchmarkUtf8mb4GeneralCI_CompareMid(b *testing.B) { @@ -85,8 +89,12 @@ func BenchmarkUtf8mb40900AICI_CompareMid(b *testing.B) { compare(b, &unicode0900AICICollator{}, middle) } +func BenchmarkUtf8mb40900Bin_CompareMid(b *testing.B) { + compare(b, &binCollator{}, middle) +} + func BenchmarkUtf8mb4Bin_CompareLong(b *testing.B) { - compare(b, &binCollator{}, long) + compare(b, &binPaddingCollator{}, long) } func BenchmarkUtf8mb4GeneralCI_CompareLong(b *testing.B) { @@ -101,8 +109,12 @@ func BenchmarkUtf8mb40900AICI_CompareLong(b *testing.B) { compare(b, &unicode0900AICICollator{}, long) } +func BenchmarkUtf8mb40900Bin_CompareLong(b *testing.B) { + compare(b, &binCollator{}, long) +} + func BenchmarkUtf8mb4Bin_KeyShort(b *testing.B) { - key(b, &binCollator{}, short) + key(b, &binPaddingCollator{}, short) } func BenchmarkUtf8mb4GeneralCI_KeyShort(b *testing.B) { @@ -117,8 +129,12 @@ func BenchmarkUtf8mb40900AICI_KeyShort(b *testing.B) { key(b, &unicode0900AICICollator{}, short) } +func BenchmarkUtf8mb40900Bin_KeyShort(b *testing.B) { + key(b, &binCollator{}, short) +} + func BenchmarkUtf8mb4Bin_KeyMid(b *testing.B) { - key(b, &binCollator{}, middle) + key(b, &binPaddingCollator{}, middle) } func BenchmarkUtf8mb4GeneralCI_KeyMid(b *testing.B) { @@ -133,8 +149,12 @@ func BenchmarkUtf8mb40900AICI_KeyMid(b *testing.B) { key(b, &unicode0900AICICollator{}, middle) } +func BenchmarkUtf8mb40900Bin_KeyMid(b *testing.B) { + key(b, &binCollator{}, middle) +} + func BenchmarkUtf8mb4Bin_KeyLong(b *testing.B) { - key(b, &binCollator{}, long) + key(b, &binPaddingCollator{}, long) } func BenchmarkUtf8mb4GeneralCI_KeyLong(b *testing.B) { @@ -148,3 +168,7 @@ func BenchmarkUtf8mb4UnicodeCI_KeyLong(b *testing.B) { func BenchmarkUtf8mb40900AICI_KeyLong(b *testing.B) { key(b, &unicode0900AICICollator{}, long) } + +func BenchmarkUtf8mb40900Bin_KeyLong(b *testing.B) { + key(b, &binCollator{}, long) +} diff --git a/util/collate/collate_test.go b/util/collate/collate_test.go index c70e8f05ca71d..34ab6bc0cdce3 100644 --- a/util/collate/collate_test.go +++ b/util/collate/collate_test.go @@ -55,28 +55,28 @@ func testKeyTable(t *testing.T, collations []string, tests []keyTable) { func TestUTF8CollatorCompare(t *testing.T) { SetNewCollationEnabledForTest(true) defer SetNewCollationEnabledForTest(false) - collations := []string{"binary", "utf8mb4_bin", "utf8mb4_general_ci", "utf8mb4_unicode_ci", "utf8mb4_0900_ai_ci", "gbk_bin", "gbk_chinese_ci"} + collations := []string{"binary", "utf8mb4_bin", "utf8mb4_general_ci", "utf8mb4_unicode_ci", "utf8mb4_0900_ai_ci", "utf8mb4_0900_bin", "gbk_bin", "gbk_chinese_ci"} tests := []compareTable{ - {"a", "b", []int{-1, -1, -1, -1, -1, -1, -1}}, - {"a", "A", []int{1, 1, 0, 0, 0, 1, 0}}, - {"Γ€", "A", []int{1, 1, 0, 0, 0, -1, -1}}, - {"abc", "abc", []int{0, 0, 0, 0, 0, 0, 0}}, - {"abc", "ab", []int{1, 1, 1, 1, 1, 1, 1}}, - {"😜", "πŸ˜ƒ", []int{1, 1, 0, 0, 1, 0, 0}}, - {"a", "a ", []int{-1, 0, 0, 0, -1, 0, 0}}, - {"a ", "a ", []int{-1, 0, 0, 0, -1, 0, 0}}, - {"a\t", "a", []int{1, 1, 1, 1, 1, 1, 1}}, - {"ß", "s", []int{1, 1, 0, 1, 1, -1, -1}}, - {"ß", "ss", []int{1, 1, -1, 0, 0, -1, -1}}, - {"ε•Š", "吧", []int{1, 1, 1, 1, 1, -1, -1}}, - {"δΈ­ζ–‡", "汉字", []int{-1, -1, -1, -1, -1, 1, 1}}, - {"Γ¦", "ae", []int{1, 1, 1, 1, 0, -1, -1}}, - {"Å", "A", []int{1, 1, 1, 0, 0, 1, 1}}, - {"Γ…", "A", []int{1, 1, 0, 0, 0, -1, -1}}, - {"\U0001730F", "ε•Š", []int{1, 1, 1, 1, -1, -1, -1}}, - {"κ°€", "㉑", []int{1, 1, 1, 1, -1, 0, 0}}, - {"갟", "감1", []int{1, 1, 1, 1, 1, -1, -1}}, - {"\U000FFFFE", "\U000FFFFF", []int{-1, -1, 0, 0, -1, 0, 0}}, + {"a", "b", []int{-1, -1, -1, -1, -1, -1, -1, -1}}, + {"a", "A", []int{1, 1, 0, 0, 0, 1, 1, 0}}, + {"Γ€", "A", []int{1, 1, 0, 0, 0, 1, -1, -1}}, + {"abc", "abc", []int{0, 0, 0, 0, 0, 0, 0, 0}}, + {"abc", "ab", []int{1, 1, 1, 1, 1, 1, 1, 1}}, + {"😜", "πŸ˜ƒ", []int{1, 1, 0, 0, 1, 1, 0, 0}}, + {"a", "a ", []int{-1, 0, 0, 0, -1, -1, 0, 0}}, + {"a ", "a ", []int{-1, 0, 0, 0, -1, -1, 0, 0}}, + {"a\t", "a", []int{1, 1, 1, 1, 1, 1, 1, 1}}, + {"ß", "s", []int{1, 1, 0, 1, 1, 1, -1, -1}}, + {"ß", "ss", []int{1, 1, -1, 0, 0, 1, -1, -1}}, + {"ε•Š", "吧", []int{1, 1, 1, 1, 1, 1, -1, -1}}, + {"δΈ­ζ–‡", "汉字", []int{-1, -1, -1, -1, -1, -1, 1, 1}}, + {"Γ¦", "ae", []int{1, 1, 1, 1, 0, 1, -1, -1}}, + {"Å", "A", []int{1, 1, 1, 0, 0, 1, 1, 1}}, + {"Γ…", "A", []int{1, 1, 0, 0, 0, 1, -1, -1}}, + {"\U0001730F", "ε•Š", []int{1, 1, 1, 1, -1, 1, -1, -1}}, + {"κ°€", "㉑", []int{1, 1, 1, 1, -1, 1, 0, 0}}, + {"갟", "감1", []int{1, 1, 1, 1, 1, 1, -1, -1}}, + {"\U000FFFFE", "\U000FFFFF", []int{-1, -1, 0, 0, -1, -1, 0, 0}}, } testCompareTable(t, collations, tests) } @@ -84,26 +84,28 @@ func TestUTF8CollatorCompare(t *testing.T) { func TestUTF8CollatorKey(t *testing.T) { SetNewCollationEnabledForTest(true) defer SetNewCollationEnabledForTest(false) - collations := []string{"binary", "utf8mb4_bin", "utf8mb4_general_ci", "utf8mb4_unicode_ci", "utf8mb4_0900_ai_ci", "gbk_bin", "gbk_chinese_ci"} + collations := []string{"binary", "utf8mb4_bin", "utf8mb4_general_ci", "utf8mb4_unicode_ci", "utf8mb4_0900_ai_ci", "utf8mb4_0900_bin", "gbk_bin", "gbk_chinese_ci"} tests := []keyTable{ - {"a", [][]byte{{0x61}, {0x61}, {0x0, 0x41}, {0x0E, 0x33}, {0x1C, 0x47}, {0x61}, {0x41}}}, - {"A", [][]byte{{0x41}, {0x41}, {0x0, 0x41}, {0x0E, 0x33}, {0x1C, 0x47}, {0x41}, {0x41}}}, + {"a", [][]byte{{0x61}, {0x61}, {0x0, 0x41}, {0x0E, 0x33}, {0x1C, 0x47}, {0x61}, {0x61}, {0x41}}}, + {"A", [][]byte{{0x41}, {0x41}, {0x0, 0x41}, {0x0E, 0x33}, {0x1C, 0x47}, {0x41}, {0x41}, {0x41}}}, {"Foo Β© bar πŒ† baz β˜ƒ qux", [][]byte{ {0x46, 0x6f, 0x6f, 0x20, 0xc2, 0xa9, 0x20, 0x62, 0x61, 0x72, 0x20, 0xf0, 0x9d, 0x8c, 0x86, 0x20, 0x62, 0x61, 0x7a, 0x20, 0xe2, 0x98, 0x83, 0x20, 0x71, 0x75, 0x78}, {0x46, 0x6f, 0x6f, 0x20, 0xc2, 0xa9, 0x20, 0x62, 0x61, 0x72, 0x20, 0xf0, 0x9d, 0x8c, 0x86, 0x20, 0x62, 0x61, 0x7a, 0x20, 0xe2, 0x98, 0x83, 0x20, 0x71, 0x75, 0x78}, {0x0, 0x46, 0x0, 0x4f, 0x0, 0x4f, 0x0, 0x20, 0x0, 0xa9, 0x0, 0x20, 0x0, 0x42, 0x0, 0x41, 0x0, 0x52, 0x0, 0x20, 0xff, 0xfd, 0x0, 0x20, 0x0, 0x42, 0x0, 0x41, 0x0, 0x5a, 0x0, 0x20, 0x26, 0x3, 0x0, 0x20, 0x0, 0x51, 0x0, 0x55, 0x0, 0x58}, {0x0E, 0xB9, 0x0F, 0x82, 0x0F, 0x82, 0x02, 0x09, 0x02, 0xC5, 0x02, 0x09, 0x0E, 0x4A, 0x0E, 0x33, 0x0F, 0xC0, 0x02, 0x09, 0xFF, 0xFD, 0x02, 0x09, 0x0E, 0x4A, 0x0E, 0x33, 0x10, 0x6A, 0x02, 0x09, 0x06, 0xFF, 0x02, 0x09, 0x0F, 0xB4, 0x10, 0x1F, 0x10, 0x5A}, {0x1c, 0xe5, 0x1d, 0xdd, 0x1d, 0xdd, 0x2, 0x9, 0x5, 0x84, 0x2, 0x9, 0x1c, 0x60, 0x1c, 0x47, 0x1e, 0x33, 0x2, 0x9, 0xe, 0xf0, 0x2, 0x9, 0x1c, 0x60, 0x1c, 0x47, 0x1f, 0x21, 0x2, 0x9, 0x9, 0x1b, 0x2, 0x9, 0x1e, 0x21, 0x1e, 0xb5, 0x1e, 0xff}, + {0x46, 0x6f, 0x6f, 0x20, 0xc2, 0xa9, 0x20, 0x62, 0x61, 0x72, 0x20, 0xf0, 0x9d, 0x8c, 0x86, 0x20, 0x62, 0x61, 0x7a, 0x20, 0xe2, 0x98, 0x83, 0x20, 0x71, 0x75, 0x78}, {0x46, 0x6f, 0x6f, 0x20, 0x3f, 0x20, 0x62, 0x61, 0x72, 0x20, 0x3f, 0x20, 0x62, 0x61, 0x7a, 0x20, 0x3f, 0x20, 0x71, 0x75, 0x78}, {0x46, 0x4f, 0x4f, 0x20, 0x3f, 0x20, 0x42, 0x41, 0x52, 0x20, 0x3f, 0x20, 0x42, 0x41, 0x5a, 0x20, 0x3f, 0x20, 0x51, 0x55, 0x58}, }}, - {"a ", [][]byte{{0x61, 0x20}, {0x61}, {0x0, 0x41}, {0x0E, 0x33}, {0x1c, 0x47, 0x2, 0x9}, {0x61}, {0x41}}}, + {"a ", [][]byte{{0x61, 0x20}, {0x61}, {0x0, 0x41}, {0x0E, 0x33}, {0x1c, 0x47, 0x2, 0x9}, {0x61, 0x20}, {0x61}, {0x41}}}, {"ο·»", [][]byte{ {0xEF, 0xB7, 0xBB}, {0xEF, 0xB7, 0xBB}, {0xFD, 0xFB}, {0x13, 0x5E, 0x13, 0xAB, 0x02, 0x09, 0x13, 0x5E, 0x13, 0xAB, 0x13, 0x50, 0x13, 0xAB, 0x13, 0xB7}, {0x23, 0x25, 0x23, 0x9c, 0x2, 0x9, 0x23, 0x25, 0x23, 0x9c, 0x23, 0xb, 0x23, 0x9c, 0x23, 0xb1}, + {0xEF, 0xB7, 0xBB}, {0x3f}, {0x3F}, }}, @@ -113,6 +115,7 @@ func TestUTF8CollatorKey(t *testing.T) { {0x4E, 0x2D, 0x65, 0x87}, {0xFB, 0x40, 0xCE, 0x2D, 0xFB, 0x40, 0xE5, 0x87}, {0xFB, 0x40, 0xCE, 0x2D, 0xfB, 0x40, 0xE5, 0x87}, + {0xE4, 0xB8, 0xAD, 0xE6, 0x96, 0x87}, {0xD6, 0xD0, 0xCE, 0xC4}, {0xD3, 0x21, 0xC1, 0xAD}, }}, @@ -122,6 +125,7 @@ func TestUTF8CollatorKey(t *testing.T) { {0xac, 0x1f, 0xac, 0x10, 0x0, 0x31}, {0xfb, 0xc1, 0xac, 0x1f, 0xfb, 0xc1, 0xac, 0x10, 0xe, 0x2a}, {0x3b, 0xf5, 0x3c, 0x74, 0x3c, 0xd3, 0x3b, 0xf5, 0x3c, 0x73, 0x3c, 0xe0, 0x1c, 0x3e}, + {0xea, 0xb0, 0x9f, 0xea, 0xb0, 0x90, 0x31}, {0x3f, 0x3f, 0x31}, {0x3f, 0x3f, 0x31}, }}, @@ -131,6 +135,7 @@ func TestUTF8CollatorKey(t *testing.T) { {0xff, 0xfd, 0xff, 0xfd}, {0xff, 0xfd, 0xff, 0xfd}, {0xfb, 0xdf, 0xff, 0xfe, 0xfb, 0xdf, 0xff, 0xff}, + {0xf3, 0xbf, 0xbf, 0xbe, 0xf3, 0xbf, 0xbf, 0xbf}, {0x3f, 0x3f}, {0x3f, 0x3f}, }}, @@ -171,6 +176,7 @@ func TestGetCollator(t *testing.T) { require.IsType(t, &unicodeCICollator{}, GetCollator("utf8_unicode_ci")) require.IsType(t, &zhPinyinTiDBASCSCollator{}, GetCollator("utf8mb4_zh_pinyin_tidb_as_cs")) require.IsType(t, &unicode0900AICICollator{}, GetCollator("utf8mb4_0900_ai_ci")) + require.IsType(t, &binCollator{}, GetCollator("utf8mb4_0900_bin")) require.IsType(t, &binPaddingCollator{}, GetCollator("default_test")) require.IsType(t, &binCollator{}, GetCollatorByID(63)) require.IsType(t, &binPaddingCollator{}, GetCollatorByID(46)) @@ -201,6 +207,7 @@ func TestGetCollator(t *testing.T) { require.IsType(t, &binCollator{}, GetCollatorByID(33)) require.IsType(t, &binCollator{}, GetCollatorByID(224)) require.IsType(t, &binCollator{}, GetCollatorByID(255)) + require.IsType(t, &binCollator{}, GetCollatorByID(309)) require.IsType(t, &binCollator{}, GetCollatorByID(192)) require.IsType(t, &binCollator{}, GetCollatorByID(2048)) require.IsType(t, &binCollator{}, GetCollatorByID(9999)) From dd593b2e032b0aa92ad1cdeaf9666f156d37a40f Mon Sep 17 00:00:00 2001 From: djshow832 Date: Tue, 22 Aug 2023 16:30:34 +0800 Subject: [PATCH 31/32] session, sessionctx: consider the dependency of sysvars for session states (#46244) close pingcap/tidb#46214 --- session/session.go | 5 +++- .../sessionstates/session_states_test.go | 22 +++++++++++++++ sessionctx/variable/sysvar.go | 8 +++--- sessionctx/variable/variable.go | 20 ++++++++++++++ sessionctx/variable/variable_test.go | 27 +++++++++++++++++++ 5 files changed, 77 insertions(+), 5 deletions(-) diff --git a/session/session.go b/session/session.go index 5d74b720b1cd9..ffb343aeb8a00 100644 --- a/session/session.go +++ b/session/session.go @@ -4315,7 +4315,10 @@ func (s *session) DecodeSessionStates(ctx context.Context, } // Decode session variables. - for name, val := range sessionStates.SystemVars { + names := variable.OrderByDependency(sessionStates.SystemVars) + // Some variables must be set before others, e.g. tidb_enable_noop_functions should be before noop variables. + for _, name := range names { + val := sessionStates.SystemVars[name] // Experimental system variables may change scope, data types, or even be removed. // We just ignore the errors and continue. if err := s.sessionVars.SetSystemVar(name, val); err != nil { diff --git a/sessionctx/sessionstates/session_states_test.go b/sessionctx/sessionstates/session_states_test.go index 66ad5f7bc7f06..bcf0da7561dd4 100644 --- a/sessionctx/sessionstates/session_states_test.go +++ b/sessionctx/sessionstates/session_states_test.go @@ -171,6 +171,28 @@ func TestSystemVars(t *testing.T) { checkStmt: "select rand()", expectedValue: "0.11641535266900002", }, + { + // tidb_enforce_mpp depends on tidb_allow_mpp. + stmts: []string{ + "set @@global.tidb_allow_mpp=0", + "set @@tidb_allow_mpp=1", + "set @@tidb_enforce_mpp=1", + }, + inSessionStates: true, + varName: variable.TiDBEnforceMPPExecution, + expectedValue: "1", + }, + { + // tx_read_only depends on tidb_enable_noop_functions. + stmts: []string{ + "set @@global.tidb_enable_noop_functions=0", + "set @@tidb_enable_noop_functions=1", + "set @@tx_read_only=1", + }, + inSessionStates: true, + varName: variable.TxReadOnly, + expectedValue: "1", + }, } if !sem.IsEnabled() { diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 9a31b9eb15181..5292d6badbfdb 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -678,7 +678,7 @@ var defaultSysVars = []*SysVar{ (*SetPDClientDynamicOption.Load())(TiDBEnableTSOFollowerProxy, val) return nil }}, - {Scope: ScopeGlobal, Name: TiDBEnableLocalTxn, Value: BoolToOnOff(DefTiDBEnableLocalTxn), Hidden: true, Type: TypeBool, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { + {Scope: ScopeGlobal, Name: TiDBEnableLocalTxn, Value: BoolToOnOff(DefTiDBEnableLocalTxn), Hidden: true, Type: TypeBool, Depended: true, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { return BoolToOnOff(EnableLocalTxn.Load()), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { oldVal := EnableLocalTxn.Load() @@ -803,7 +803,7 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeGlobal, Name: TiDBEnableTelemetry, Value: BoolToOnOff(DefTiDBEnableTelemetry), Type: TypeBool}, - {Scope: ScopeGlobal, Name: TiDBEnableHistoricalStats, Value: On, Type: TypeBool}, + {Scope: ScopeGlobal, Name: TiDBEnableHistoricalStats, Value: On, Type: TypeBool, Depended: true}, /* tikv gc metrics */ {Scope: ScopeGlobal, Name: TiDBGCEnable, Value: On, Type: TypeBool, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return getTiDBTableValue(s, "tikv_gc_enable", On) @@ -1549,7 +1549,7 @@ var defaultSysVars = []*SysVar{ }}, {Scope: ScopeGlobal | ScopeSession, Name: BlockEncryptionMode, Value: "aes-128-ecb", Type: TypeEnum, PossibleValues: []string{"aes-128-ecb", "aes-192-ecb", "aes-256-ecb", "aes-128-cbc", "aes-192-cbc", "aes-256-cbc", "aes-128-ofb", "aes-192-ofb", "aes-256-ofb", "aes-128-cfb", "aes-192-cfb", "aes-256-cfb"}}, /* TiDB specific variables */ - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowMPPExecution, Type: TypeBool, Value: BoolToOnOff(DefTiDBAllowMPPExecution), SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowMPPExecution, Type: TypeBool, Value: BoolToOnOff(DefTiDBAllowMPPExecution), Depended: true, SetSession: func(s *SessionVars, val string) error { s.allowMPPExecution = TiDBOptOn(val) return nil }}, @@ -1893,7 +1893,7 @@ var defaultSysVars = []*SysVar{ s.TiDBOptJoinReorderThreshold = tidbOptPositiveInt32(val, DefTiDBOptJoinReorderThreshold) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNoopFuncs, Value: DefTiDBEnableNoopFuncs, Type: TypeEnum, PossibleValues: []string{Off, On, Warn}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNoopFuncs, Value: DefTiDBEnableNoopFuncs, Type: TypeEnum, PossibleValues: []string{Off, On, Warn}, Depended: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { // The behavior is very weird if someone can turn TiDBEnableNoopFuncs OFF, but keep any of the following on: // TxReadOnly, TransactionReadOnly, OfflineMode, SuperReadOnly, serverReadOnly, SQLAutoIsNull // To prevent this strange position, prevent setting to OFF when any of these sysVars are ON of the same scope. diff --git a/sessionctx/variable/variable.go b/sessionctx/variable/variable.go index 38457a4f5687f..ea570963d4b01 100644 --- a/sessionctx/variable/variable.go +++ b/sessionctx/variable/variable.go @@ -155,6 +155,10 @@ type SysVar struct { // GetStateValue gets the value for session states, which is used for migrating sessions. // We need a function to override GetSession sometimes, because GetSession may not return the real value. GetStateValue func(*SessionVars) (string, bool, error) + // Depended indicates whether other variables depend on this one. That is, if this one is not correctly set, + // another variable cannot be set either. + // This flag is used to decide the order to replay session variables. + Depended bool // skipInit defines if the sysvar should be loaded into the session on init. // This is only important to set for sysvars that include session scope, // since global scoped sysvars are not-applicable. @@ -614,6 +618,22 @@ func GetSysVars() map[string]*SysVar { return m } +// OrderByDependency orders the vars by dependency. The depended sys vars are in the front. +// Unknown sys vars are treated as not depended. +func OrderByDependency(names map[string]string) []string { + depended, notDepended := make([]string, 0, len(names)), make([]string, 0, len(names)) + sysVarsLock.RLock() + defer sysVarsLock.RUnlock() + for name := range names { + if sv, ok := sysVars[name]; ok && sv.Depended { + depended = append(depended, name) + } else { + notDepended = append(notDepended, name) + } + } + return append(depended, notDepended...) +} + func init() { sysVars = make(map[string]*SysVar) for _, v := range defaultSysVars { diff --git a/sessionctx/variable/variable_test.go b/sessionctx/variable/variable_test.go index 0c61d4e85b890..89da5597ee8fd 100644 --- a/sessionctx/variable/variable_test.go +++ b/sessionctx/variable/variable_test.go @@ -19,11 +19,13 @@ import ( "encoding/json" "fmt" "runtime" + "slices" "strings" "testing" "time" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/types" @@ -687,3 +689,28 @@ func TestTimeValidationWithTimezone(t *testing.T) { require.NoError(t, err) require.Equal(t, "23:59 +0800", val) } + +func TestOrderByDependency(t *testing.T) { + // Some other exceptions: + // - tidb_snapshot and tidb_read_staleness can not be set at the same time. It doesn't affect dependency. + vars := map[string]string{ + "unknown": "1", + TxReadOnly: "1", + SQLAutoIsNull: "1", + TiDBEnableNoopFuncs: "1", + TiDBEnforceMPPExecution: "1", + TiDBAllowMPPExecution: "1", + TiDBTxnScope: kv.LocalTxnScope, + TiDBEnableLocalTxn: "1", + TiDBEnablePlanReplayerContinuousCapture: "1", + TiDBEnableHistoricalStats: "1", + } + names := OrderByDependency(vars) + require.Greater(t, slices.Index(names, TxReadOnly), slices.Index(names, TiDBEnableNoopFuncs)) + require.Greater(t, slices.Index(names, SQLAutoIsNull), slices.Index(names, TiDBEnableNoopFuncs)) + require.Greater(t, slices.Index(names, TiDBEnforceMPPExecution), slices.Index(names, TiDBAllowMPPExecution)) + // Depended variables below are global variables, so actually it doesn't matter. + require.Greater(t, slices.Index(names, TiDBTxnScope), slices.Index(names, TiDBEnableLocalTxn)) + require.Greater(t, slices.Index(names, TiDBEnablePlanReplayerContinuousCapture), slices.Index(names, TiDBEnableHistoricalStats)) + require.Contains(t, names, "unknown") +} From cb248a152a259cc585448d90b474fb29e63cde8c Mon Sep 17 00:00:00 2001 From: lance6716 Date: Tue, 22 Aug 2023 17:47:03 +0800 Subject: [PATCH 32/32] lightning: "no leader" should be a retryable error (#46300) close pingcap/tidb#45673 --- br/pkg/lightning/backend/local/region_job.go | 8 ++++++-- br/pkg/lightning/common/errors.go | 1 + br/pkg/lightning/common/retry.go | 1 + br/tests/lightning_local_backend/config.toml | 1 + br/tests/lightning_local_backend/run.sh | 2 +- errors.toml | 5 +++++ 6 files changed, 15 insertions(+), 3 deletions(-) diff --git a/br/pkg/lightning/backend/local/region_job.go b/br/pkg/lightning/backend/local/region_job.go index a07f6fa3c9bd6..5de40cca0e195 100644 --- a/br/pkg/lightning/backend/local/region_job.go +++ b/br/pkg/lightning/backend/local/region_job.go @@ -408,6 +408,11 @@ func (local *Backend) writeToTiKV(ctx context.Context, j *regionJob) error { } } + failpoint.Inject("NoLeader", func() { + log.FromContext(ctx).Warn("enter failpoint NoLeader") + leaderPeerMetas = nil + }) + // if there is not leader currently, we don't forward the stage to wrote and let caller // handle the retry. if len(leaderPeerMetas) == 0 { @@ -415,8 +420,7 @@ func (local *Backend) writeToTiKV(ctx context.Context, j *regionJob) error { logutil.Region(region), logutil.Leader(j.region.Leader), zap.Uint64("leader_id", leaderID), logutil.SSTMeta(meta), zap.Int64("kv_pairs", totalCount), zap.Int64("total_bytes", totalSize)) - return errors.Errorf("write to tikv with no leader returned, region '%d', leader: %d", - region.Id, leaderID) + return common.ErrNoLeader.GenWithStackByArgs(region.Id, leaderID) } takeTime := time.Since(begin) diff --git a/br/pkg/lightning/common/errors.go b/br/pkg/lightning/common/errors.go index 9ea4e7b2af63d..109f72755720f 100644 --- a/br/pkg/lightning/common/errors.go +++ b/br/pkg/lightning/common/errors.go @@ -83,6 +83,7 @@ var ( ErrKVReadIndexNotReady = errors.Normalize("read index not ready", errors.RFCCodeText("Lightning:KV:ReadIndexNotReady")) ErrKVIngestFailed = errors.Normalize("ingest tikv failed", errors.RFCCodeText("Lightning:KV:ErrKVIngestFailed")) ErrKVRaftProposalDropped = errors.Normalize("raft proposal dropped", errors.RFCCodeText("Lightning:KV:ErrKVRaftProposalDropped")) + ErrNoLeader = errors.Normalize("write to tikv with no leader returned, region '%d', leader: %d", errors.RFCCodeText("Lightning:KV:ErrNoLeader")) ErrUnknownBackend = errors.Normalize("unknown backend %s", errors.RFCCodeText("Lightning:Restore:ErrUnknownBackend")) ErrCheckLocalFile = errors.Normalize("cannot find local file for table: %s engineDir: %s", errors.RFCCodeText("Lightning:Restore:ErrCheckLocalFile")) diff --git a/br/pkg/lightning/common/retry.go b/br/pkg/lightning/common/retry.go index f6db6cda86407..c3bb979a9bd32 100644 --- a/br/pkg/lightning/common/retry.go +++ b/br/pkg/lightning/common/retry.go @@ -70,6 +70,7 @@ func IsRetryableError(err error) bool { var retryableErrorIDs = map[errors.ErrorID]struct{}{ ErrKVEpochNotMatch.ID(): {}, ErrKVNotLeader.ID(): {}, + ErrNoLeader.ID(): {}, ErrKVRegionNotFound.ID(): {}, // common.ErrKVServerIsBusy is a little duplication with tmysql.ErrTiKVServerBusy // it's because the response of sst.ingest gives us a sst.IngestResponse which doesn't contain error code, diff --git a/br/tests/lightning_local_backend/config.toml b/br/tests/lightning_local_backend/config.toml index 46ca06e09b4ab..73c54882430c7 100644 --- a/br/tests/lightning_local_backend/config.toml +++ b/br/tests/lightning_local_backend/config.toml @@ -1,5 +1,6 @@ [lightning] table-concurrency = 1 +index-concurrency = 1 [checkpoint] enable = true diff --git a/br/tests/lightning_local_backend/run.sh b/br/tests/lightning_local_backend/run.sh index 3fdde7328da13..8e8f9b8080fb5 100755 --- a/br/tests/lightning_local_backend/run.sh +++ b/br/tests/lightning_local_backend/run.sh @@ -56,7 +56,7 @@ check_contains 'sum(c): 46' run_sql 'DROP DATABASE cpeng;' rm -f "/tmp/tidb_lightning_checkpoint_local_backend_test.pb" -export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/lightning/backend/local/FailIngestMeta=2*return("epochnotmatch")' +export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/lightning/backend/local/FailIngestMeta=2*return("epochnotmatch");github.com/pingcap/tidb/br/pkg/lightning/backend/local/NoLeader=1*return()' run_lightning --backend local --enable-checkpoint=1 --log-file "$TEST_DIR/lightning-local.log" --config "$CUR/config.toml" diff --git a/errors.toml b/errors.toml index 6a2ca3036e5b7..afb49d8cd9f76 100644 --- a/errors.toml +++ b/errors.toml @@ -406,6 +406,11 @@ error = ''' raft proposal dropped ''' +["Lightning:KV:ErrNoLeader"] +error = ''' +write to tikv with no leader returned, region '%d', leader: %d +''' + ["Lightning:KV:NotLeader"] error = ''' not leader