diff --git a/.github/dependabot.yml b/.github/dependabot.yml index bde24b721461e..b2980c65d8372 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -11,4 +11,7 @@ updates: - dependency-name: "golang.org/*" - dependency-name: "github.com/golangci/golangci-lint" open-pull-requests-limit: 2 - + - package-ecosystem: "gomod" + directory: "/parser" + schedule: + interval: "weekly" diff --git a/DEPS.bzl b/DEPS.bzl index 8d2ef6d2cb372..47022748fa1ef 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -210,8 +210,8 @@ def go_deps(): name = "com_github_ashanbrown_forbidigo", build_file_proto_mode = "disable", importpath = "github.com/ashanbrown/forbidigo", - sum = "h1:VkYIwb/xxdireGAdJNZoo24O4lmnEWkactplBlWTShc=", - version = "v1.3.0", + sum = "h1:spdPbupaSqtWORq1Q4eHBoPBmHtwVyLKwaedbSLc5Sw=", + version = "v1.4.0", ) go_repository( name = "com_github_ashanbrown_makezero", @@ -364,8 +364,8 @@ def go_deps(): name = "com_github_bombsimon_wsl_v3", build_file_proto_mode = "disable", importpath = "github.com/bombsimon/wsl/v3", - sum = "h1:Mka/+kRLoQJq7g2rggtgQsjuI/K5Efd87WX96EWFxjM=", - version = "v3.3.0", + sum = "h1:RkSxjT3tmlptwfgEgTgU+KYKLI35p/tviNXNXiL2aNU=", + version = "v3.4.0", ) go_repository( name = "com_github_breeswish_gin_jwt_v2", @@ -778,8 +778,8 @@ def go_deps(): name = "com_github_daixiang0_gci", build_file_proto_mode = "disable", importpath = "github.com/daixiang0/gci", - sum = "h1:t8XZ0vK6l0pwPoOmoGyqW2NwQlvbpAQNVvu/GRBgykM=", - version = "v0.9.0", + sum = "h1:jBrwBmBZTDsGsXiaCTLIe9diotp1X4X64zodFrh7l+c=", + version = "v0.9.1", ) go_repository( @@ -1213,8 +1213,8 @@ def go_deps(): name = "com_github_go_critic_go_critic", build_file_proto_mode = "disable", importpath = "github.com/go-critic/go-critic", - sum = "h1:fDaR/5GWURljXwF8Eh31T2GZNz9X4jeboS912mWF8Uo=", - version = "v0.6.5", + sum = "h1:1evPrElnLQ2LZtJfmNDzlieDhjnq36SLgNzisx06oPM=", + version = "v0.6.7", ) go_repository( name = "com_github_go_echarts_go_echarts", @@ -1378,8 +1378,8 @@ def go_deps(): name = "com_github_go_toolsmith_astcast", build_file_proto_mode = "disable", importpath = "github.com/go-toolsmith/astcast", - sum = "h1:JojxlmI6STnFVG9yOImLeGREv8W2ocNUM+iOhR6jE7g=", - version = "v1.0.0", + sum = "h1:+JN9xZV1A+Re+95pgnMgDboWNVnIMMQXwfBwLRPgSC8=", + version = "v1.1.0", ) go_repository( name = "com_github_go_toolsmith_astcopy", @@ -1392,36 +1392,36 @@ def go_deps(): name = "com_github_go_toolsmith_astequal", build_file_proto_mode = "disable", importpath = "github.com/go-toolsmith/astequal", - sum = "h1:+LVdyRatFS+XO78SGV4I3TCEA0AC7fKEGma+fH+674o=", - version = "v1.0.3", + sum = "h1:kHKm1AWqClYn15R0K1KKE4RG614D46n+nqUQ06E1dTw=", + version = "v1.1.0", ) go_repository( name = "com_github_go_toolsmith_astfmt", build_file_proto_mode = "disable", importpath = "github.com/go-toolsmith/astfmt", - sum = "h1:A0vDDXt+vsvLEdbMFJAUBI/uTbRw1ffOPnxsILnFL6k=", - version = "v1.0.0", + sum = "h1:iJVPDPp6/7AaeLJEruMsBUlOYCmvg0MoCfJprsOmcco=", + version = "v1.1.0", ) go_repository( name = "com_github_go_toolsmith_astp", build_file_proto_mode = "disable", importpath = "github.com/go-toolsmith/astp", - sum = "h1:alXE75TXgcmupDsMK1fRAy0YUzLzqPVvBKoyWV+KPXg=", - version = "v1.0.0", + sum = "h1:dXPuCl6u2llURjdPLLDxJeZInAeZ0/eZwFJmqZMnpQA=", + version = "v1.1.0", ) go_repository( name = "com_github_go_toolsmith_strparse", build_file_proto_mode = "disable", importpath = "github.com/go-toolsmith/strparse", - sum = "h1:Vcw78DnpCAKlM20kSbAyO4mPfJn/lyYA4BJUDxe2Jb4=", - version = "v1.0.0", + sum = "h1:GAioeZUK9TGxnLS+qfdqNbA4z0SSm5zVNtCQiyP2Bvw=", + version = "v1.1.0", ) go_repository( name = "com_github_go_toolsmith_typep", build_file_proto_mode = "disable", importpath = "github.com/go-toolsmith/typep", - sum = "h1:8xdsa1+FSIH/RhEkgnD1j2CJOy5mNllW1Q9tRiYwvlk=", - version = "v1.0.2", + sum = "h1:fIRYDyF+JywLfqzyhdiHzRop/GQDxxNhLGQ6gFUNHus=", + version = "v1.1.0", ) go_repository( name = "com_github_go_xmlfmt_xmlfmt", @@ -1606,8 +1606,8 @@ def go_deps(): name = "com_github_golangci_golangci_lint", build_file_proto_mode = "disable", importpath = "github.com/golangci/golangci-lint", - sum = "h1:N5HD/x0ZrhJYsgKWyz7yJxxQ8JKR0Acc+FOP7QtGSAA=", - version = "v1.51.1", + sum = "h1:yIcsT1X9ZYHdSpeWXRT1ORC/FPGSqDHbHsu9uk4FK7M=", + version = "v1.51.2", ) go_repository( name = "com_github_golangci_gosec", @@ -2440,8 +2440,8 @@ def go_deps(): name = "com_github_junk1tm_musttag", build_file_proto_mode = "disable", importpath = "github.com/junk1tm/musttag", - sum = "h1:VK4L7v7lvWAhKDDx0cUJgbb0UBNipYinv8pPeHJzH9Q=", - version = "v0.4.4", + sum = "h1:d+mpJ1vn6WFEVKHwkgJiIedis1u/EawKOuUTygAUtCo=", + version = "v0.4.5", ) go_repository( @@ -3445,8 +3445,8 @@ def go_deps(): name = "com_github_polyfloyd_go_errorlint", build_file_proto_mode = "disable", importpath = "github.com/polyfloyd/go-errorlint", - sum = "h1:ZevdyEGxDoHAMQUVvdTT06hnYuKULe8TQkOmIYx6s1c=", - version = "v1.0.6", + sum = "h1:VKoEFg5yxSgJ2yFPVhxW7oGz+f8/OVcuMeNvcPIi6Eg=", + version = "v1.1.0", ) go_repository( @@ -3525,8 +3525,8 @@ def go_deps(): name = "com_github_quasilyte_go_ruleguard", build_file_proto_mode = "disable", importpath = "github.com/quasilyte/go-ruleguard", - sum = "h1:sd+abO1PEI9fkYennwzHn9kl3nqP6M5vE7FiOzZ+5CE=", - version = "v0.3.18", + sum = "h1:tfMnabXle/HzOb5Xe9CUZYWXKfkS1KwRmZyPmD9nVcc=", + version = "v0.3.19", ) go_repository( name = "com_github_quasilyte_go_ruleguard_dsl", @@ -3539,8 +3539,8 @@ def go_deps(): name = "com_github_quasilyte_gogrep", build_file_proto_mode = "disable", importpath = "github.com/quasilyte/gogrep", - sum = "h1:6Gtn2i04RD0gVyYf2/IUMTIs+qYleBt4zxDqkLTcu4U=", - version = "v0.0.0-20220828223005-86e4605de09f", + sum = "h1:eTKODPXbI8ffJMN+W2aE0+oL0z/nh8/5eNdiO34SOAo=", + version = "v0.5.0", ) go_repository( name = "com_github_quasilyte_regex_syntax", @@ -3693,8 +3693,8 @@ def go_deps(): name = "com_github_sashamelentyev_usestdlibvars", build_file_proto_mode = "disable", importpath = "github.com/sashamelentyev/usestdlibvars", - sum = "h1:GQGlReyL9Ek8DdJmwtwhHbhwHnuPfsKaprpjnrPcjxc=", - version = "v1.21.1", + sum = "h1:01h+/2Kd+NblNItNeux0veSL5cBF1jbEOPrEhDzGYq0=", + version = "v1.23.0", ) go_repository( @@ -3723,8 +3723,8 @@ def go_deps(): name = "com_github_securego_gosec_v2", build_file_proto_mode = "disable", importpath = "github.com/securego/gosec/v2", - sum = "h1:U1hfs0oBackChXA72plCYVA4cOlQ4gO+209dHiSNZbI=", - version = "v2.14.0", + sum = "h1:v4Ym7FF58/jlykYmmhZ7mTm7FQvN/setNm++0fgIAtw=", + version = "v2.15.0", ) go_repository( @@ -4101,8 +4101,8 @@ def go_deps(): name = "com_github_tikv_client_go_v2", build_file_proto_mode = "disable_global", importpath = "github.com/tikv/client-go/v2", - sum = "h1:HgU1sa4pzjduuMAMysbfAt8ebx5p2SZyPFu/DC0Ir0c=", - version = "v2.0.6-0.20230222073543-a27994e5aae2", + sum = "h1:hDbgZE2B2ZNZ4ID8E57Z/glM/xXAy11L00S554NRSx8=", + version = "v2.0.6-0.20230228054731-11d34cbd8a7f", ) go_repository( name = "com_github_tikv_pd", @@ -5888,8 +5888,8 @@ def go_deps(): name = "org_golang_x_exp_typeparams", build_file_proto_mode = "disable", importpath = "golang.org/x/exp/typeparams", - sum = "h1:Jw5wfR+h9mnIYH+OtGT2im5wV1YGGDora5vTv/aa5bE=", - version = "v0.0.0-20221208152030-732eee02a75a", + sum = "h1:6WHiuFL9FNjg8RljAaT7FNUuKDbvMqS1i5cr2OE2sLQ=", + version = "v0.0.0-20230203172020-98cc5a0785f9", ) go_repository( diff --git a/br/pkg/lightning/backend/local/BUILD.bazel b/br/pkg/lightning/backend/local/BUILD.bazel index e404b5f971c22..60ec468fb8cb6 100644 --- a/br/pkg/lightning/backend/local/BUILD.bazel +++ b/br/pkg/lightning/backend/local/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "local_unix_generic.go", "local_windows.go", "localhelper.go", + "region_job.go", ], importpath = "github.com/pingcap/tidb/br/pkg/lightning/backend/local", visibility = ["//visibility:public"], @@ -28,7 +29,6 @@ go_library( "//br/pkg/lightning/manual", "//br/pkg/lightning/metric", "//br/pkg/lightning/tikv", - "//br/pkg/lightning/worker", "//br/pkg/logutil", "//br/pkg/membuf", "//br/pkg/pdutil", @@ -40,6 +40,7 @@ go_library( "//kv", "//parser/model", "//parser/mysql", + "//store/pdtypes", "//table", "//tablecodec", "//types", @@ -93,19 +94,18 @@ go_test( "local_check_test.go", "local_test.go", "localhelper_test.go", + "region_job_test.go", ], embed = [":local"], flaky = True, shard_count = 45, deps = [ - "//br/pkg/errors", "//br/pkg/lightning/backend", "//br/pkg/lightning/backend/kv", "//br/pkg/lightning/common", "//br/pkg/lightning/glue", "//br/pkg/lightning/log", "//br/pkg/lightning/mydump", - "//br/pkg/lightning/worker", "//br/pkg/membuf", "//br/pkg/mock", "//br/pkg/pdutil", diff --git a/br/pkg/lightning/backend/local/engine.go b/br/pkg/lightning/backend/local/engine.go index 745454a372b18..395b69ece0800 100644 --- a/br/pkg/lightning/backend/local/engine.go +++ b/br/pkg/lightning/backend/local/engine.go @@ -988,7 +988,14 @@ func (e *Engine) newKVIter(ctx context.Context, opts *pebble.IterOptions) Iter { return newDupDetectIter(ctx, e.db, e.keyAdapter, opts, e.duplicateDB, logger, e.dupDetectOpt) } +// getFirstAndLastKey reads the first and last key in range [lowerBound, upperBound) +// in the engine. Empty upperBound means unbounded. func (e *Engine) getFirstAndLastKey(lowerBound, upperBound []byte) ([]byte, []byte, error) { + if len(upperBound) == 0 { + // we use empty slice for unbounded upper bound, but it means max value in pebble + // so reset to nil + upperBound = nil + } opt := &pebble.IterOptions{ LowerBound: lowerBound, UpperBound: upperBound, diff --git a/br/pkg/lightning/backend/local/engine_test.go b/br/pkg/lightning/backend/local/engine_test.go index 2b935f30bb0c2..46f96a09484c5 100644 --- a/br/pkg/lightning/backend/local/engine_test.go +++ b/br/pkg/lightning/backend/local/engine_test.go @@ -122,4 +122,9 @@ func TestGetFirstAndLastKey(t *testing.T) { require.NoError(t, err) require.Nil(t, first) require.Nil(t, last) + + first, last, err = f.getFirstAndLastKey([]byte("e"), []byte("")) + require.NoError(t, err) + require.Equal(t, []byte("e"), first) + require.Equal(t, []byte("e"), last) } diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index f915e270ac0eb..32effde381d3c 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -33,9 +33,7 @@ import ( "github.com/google/uuid" "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/errorpb" sst "github.com/pingcap/kvproto/pkg/import_sstpb" - "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" @@ -47,7 +45,6 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/manual" "github.com/pingcap/tidb/br/pkg/lightning/metric" "github.com/pingcap/tidb/br/pkg/lightning/tikv" - "github.com/pingcap/tidb/br/pkg/lightning/worker" "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/membuf" "github.com/pingcap/tidb/br/pkg/pdutil" @@ -57,6 +54,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/store/pdtypes" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -68,7 +66,6 @@ import ( tikvclient "github.com/tikv/client-go/v2/tikv" pd "github.com/tikv/pd/client" "go.uber.org/atomic" - "go.uber.org/multierr" "go.uber.org/zap" "golang.org/x/exp/slices" "golang.org/x/sync/errgroup" @@ -90,6 +87,7 @@ const ( // maxWriteAndIngestRetryTimes is the max retry times for write and ingest. // A large retry times is for tolerating tikv cluster failures. maxWriteAndIngestRetryTimes = 30 + maxRetryBackoffSecond = 30 maxRetryBackoffTime = 30 * time.Second gRPCKeepAliveTime = 10 * time.Minute @@ -241,7 +239,7 @@ func (c loggingConn) Write(b []byte) (int, error) { // so we can write it to tikv in streaming type Range struct { start []byte - end []byte + end []byte // end is always exclusive except import_sstpb.SSTMeta } type encodingBuilder struct { @@ -396,9 +394,8 @@ type local struct { localStoreDir string - rangeConcurrency *worker.Pool - ingestConcurrency *worker.Pool - batchWriteKVPairs int + workerConcurrency int + kvWriteBatchSize int checkpointEnabled bool dupeConcurrency int @@ -408,7 +405,7 @@ type local struct { localWriterMemCacheSize int64 supportMultiIngest bool - checkTiKVAvaliable bool + shouldCheckTiKV bool duplicateDetection bool duplicateDetectOpt dupDetectOpt duplicateDB *pebble.DB @@ -544,10 +541,9 @@ func NewLocalBackend( tikvCodec: tikvCodec, localStoreDir: localFile, - rangeConcurrency: worker.NewPool(ctx, rangeConcurrency, "range"), - ingestConcurrency: worker.NewPool(ctx, rangeConcurrency*2, "ingest"), + workerConcurrency: rangeConcurrency * 2, dupeConcurrency: rangeConcurrency * 2, - batchWriteKVPairs: cfg.TikvImporter.SendKVPairs, + kvWriteBatchSize: cfg.TikvImporter.SendKVPairs, checkpointEnabled: cfg.Checkpoint.Enable, maxOpenFiles: mathutil.Max(maxOpenFiles, openFilesLowerThreshold), @@ -555,7 +551,7 @@ func NewLocalBackend( localWriterMemCacheSize: int64(cfg.TikvImporter.LocalWriterMemCacheSize), duplicateDetection: duplicateDetection, duplicateDetectOpt: dupDetectOpt{duplicateDetection && cfg.TikvImporter.DuplicateResolution == config.DupeResAlgErr}, - checkTiKVAvaliable: cfg.App.CheckRequirements, + shouldCheckTiKV: cfg.App.CheckRequirements, duplicateDB: duplicateDB, keyAdapter: keyAdapter, errorMgr: errorMgr, @@ -950,311 +946,6 @@ func (local *local) getImportClient(ctx context.Context, storeID uint64) (sst.Im return local.importClientFactory.Create(ctx, storeID) } -type rangeStats struct { - count int64 - totalBytes int64 -} - -type tikvWriteResult struct { - sstMeta []*sst.SSTMeta - finishedRange Range - rangeStats rangeStats -} - -// WriteToTiKV writer engine key-value pairs to tikv and return the sst meta generated by tikv. -// we don't need to do cleanup for the pairs written to tikv if encounters an error, -// tikv will take the responsibility to do so. -func (local *local) WriteToTiKV( - ctx context.Context, - engine *Engine, - region *split.RegionInfo, - start, end []byte, - regionSplitSize int64, - regionSplitKeys int64, -) (*tikvWriteResult, error) { - failpoint.Inject("WriteToTiKVNotEnoughDiskSpace", func(_ failpoint.Value) { - failpoint.Return(nil, - errors.Errorf("The available disk of TiKV (%s) only left %d, and capacity is %d", "", 0, 0)) - }) - if local.checkTiKVAvaliable { - for _, peer := range region.Region.GetPeers() { - var e error - for i := 0; i < maxRetryTimes; i++ { - store, err := local.pdCtl.GetStoreInfo(ctx, peer.StoreId) - if err != nil { - e = err - continue - } - if store.Status.Capacity > 0 { - // The available disk percent of TiKV - ratio := store.Status.Available * 100 / store.Status.Capacity - if ratio < 10 { - return nil, errors.Errorf("The available disk of TiKV (%s) only left %d, and capacity is %d", - store.Store.Address, store.Status.Available, store.Status.Capacity) - } - } - break - } - if e != nil { - log.FromContext(ctx).Error("failed to get StoreInfo from pd http api", zap.Error(e)) - } - } - } - begin := time.Now() - regionRange := intersectRange(region.Region, Range{start: start, end: end}) - stats := rangeStats{} - - firstKey, lastKey, err := engine.getFirstAndLastKey(regionRange.start, regionRange.end) - if err != nil { - return nil, errors.Trace(err) - } - if firstKey == nil { - log.FromContext(ctx).Info("keys within region is empty, skip ingest", logutil.Key("start", start), - logutil.Key("regionStart", region.Region.StartKey), logutil.Key("end", end), - logutil.Key("regionEnd", region.Region.EndKey)) - return &tikvWriteResult{sstMeta: nil, finishedRange: regionRange, rangeStats: stats}, nil - } - firstKey = codec.EncodeBytes([]byte{}, firstKey) - lastKey = codec.EncodeBytes([]byte{}, lastKey) - - u := uuid.New() - meta := &sst.SSTMeta{ - Uuid: u[:], - RegionId: region.Region.GetId(), - RegionEpoch: region.Region.GetRegionEpoch(), - Range: &sst.Range{ - Start: firstKey, - End: lastKey, - }, - ApiVersion: local.tikvCodec.GetAPIVersion(), - } - - leaderID := region.Leader.GetId() - clients := make([]sst.ImportSST_WriteClient, 0, len(region.Region.GetPeers())) - storeIDs := make([]uint64, 0, len(region.Region.GetPeers())) - requests := make([]*sst.WriteRequest, 0, len(region.Region.GetPeers())) - for _, peer := range region.Region.GetPeers() { - cli, err := local.getImportClient(ctx, peer.StoreId) - if err != nil { - return nil, errors.Trace(err) - } - - wstream, err := cli.Write(ctx) - if err != nil { - return nil, errors.Trace(err) - } - - // Bind uuid for this write request - req := &sst.WriteRequest{ - Chunk: &sst.WriteRequest_Meta{ - Meta: meta, - }, - } - if err = wstream.Send(req); err != nil { - return nil, errors.Trace(err) - } - req.Chunk = &sst.WriteRequest_Batch{ - Batch: &sst.WriteBatch{ - CommitTs: engine.TS, - }, - } - clients = append(clients, wstream) - requests = append(requests, req) - storeIDs = append(storeIDs, peer.StoreId) - } - - bytesBuf := local.bufferPool.NewBuffer() - defer bytesBuf.Destroy() - pairs := make([]*sst.Pair, 0, local.batchWriteKVPairs) - count := 0 - size := int64(0) - totalSize := int64(0) - totalCount := int64(0) - // if region-split-size <= 96MiB, we bump the threshold a bit to avoid too many retry split - // because the range-properties is not 100% accurate - regionMaxSize := regionSplitSize - if regionSplitSize <= int64(config.SplitRegionSize) { - regionMaxSize = regionSplitSize * 4 / 3 - } - // Set a lower flush limit to make the speed of write more smooth. - flushLimit := int64(local.writeLimiter.Limit() / 10) - - flushKVs := func() error { - for i := range clients { - if err := local.writeLimiter.WaitN(ctx, storeIDs[i], int(size)); err != nil { - return errors.Trace(err) - } - requests[i].Chunk.(*sst.WriteRequest_Batch).Batch.Pairs = pairs[:count] - if err := clients[i].Send(requests[i]); err != nil { - return errors.Trace(err) - } - } - return nil - } - - opt := &pebble.IterOptions{LowerBound: regionRange.start, UpperBound: regionRange.end} - iter := engine.newKVIter(ctx, opt) - //nolint: errcheck - defer iter.Close() - - for iter.First(); iter.Valid(); iter.Next() { - kvSize := int64(len(iter.Key()) + len(iter.Value())) - // here we reuse the `*sst.Pair`s to optimize object allocation - if count < len(pairs) { - pairs[count].Key = bytesBuf.AddBytes(iter.Key()) - pairs[count].Value = bytesBuf.AddBytes(iter.Value()) - } else { - pair := &sst.Pair{ - Key: bytesBuf.AddBytes(iter.Key()), - Value: bytesBuf.AddBytes(iter.Value()), - } - pairs = append(pairs, pair) - } - count++ - totalCount++ - size += kvSize - totalSize += kvSize - - if count >= local.batchWriteKVPairs || size >= flushLimit { - if err := flushKVs(); err != nil { - return nil, errors.Trace(err) - } - count = 0 - size = 0 - bytesBuf.Reset() - } - if totalSize >= regionMaxSize || totalCount >= regionSplitKeys { - break - } - } - - if iter.Error() != nil { - return nil, errors.Trace(iter.Error()) - } - - if count > 0 { - if err := flushKVs(); err != nil { - return nil, errors.Trace(err) - } - count = 0 - size = 0 - bytesBuf.Reset() - } - - var leaderPeerMetas []*sst.SSTMeta - for i, wStream := range clients { - resp, closeErr := wStream.CloseAndRecv() - if closeErr != nil { - return nil, errors.Trace(closeErr) - } - if resp.Error != nil { - return nil, errors.New(resp.Error.Message) - } - if leaderID == region.Region.Peers[i].GetId() { - leaderPeerMetas = resp.Metas - log.FromContext(ctx).Debug("get metas after write kv stream to tikv", zap.Reflect("metas", leaderPeerMetas)) - } - } - - // if there is not leader currently, we should directly return an error - if len(leaderPeerMetas) == 0 { - log.FromContext(ctx).Warn("write to tikv no leader", logutil.Region(region.Region), logutil.Leader(region.Leader), - zap.Uint64("leader_id", leaderID), logutil.SSTMeta(meta), - zap.Int64("kv_pairs", totalCount), zap.Int64("total_bytes", totalSize)) - return nil, errors.Errorf("write to tikv with no leader returned, region '%d', leader: %d", - region.Region.Id, leaderID) - } - - log.FromContext(ctx).Debug("write to kv", zap.Reflect("region", region), zap.Uint64("leader", leaderID), - zap.Reflect("meta", meta), zap.Reflect("return metas", leaderPeerMetas), - zap.Int64("kv_pairs", totalCount), zap.Int64("total_bytes", totalSize), - zap.Int64("buf_size", bytesBuf.TotalSize()), - zap.Stringer("takeTime", time.Since(begin))) - - finishedRange := regionRange - if iter.Valid() && iter.Next() { - firstKey := append([]byte{}, iter.Key()...) - finishedRange = Range{start: regionRange.start, end: firstKey} - log.FromContext(ctx).Info("write to tikv partial finish", zap.Int64("count", totalCount), - zap.Int64("size", totalSize), logutil.Key("startKey", regionRange.start), logutil.Key("endKey", regionRange.end), - logutil.Key("remainStart", firstKey), logutil.Key("remainEnd", regionRange.end), - logutil.Region(region.Region), logutil.Leader(region.Leader)) - } - stats.count = totalCount - stats.totalBytes = totalSize - - return &tikvWriteResult{ - sstMeta: leaderPeerMetas, - finishedRange: finishedRange, - rangeStats: stats, - }, nil -} - -func (local *local) Ingest(ctx context.Context, metas []*sst.SSTMeta, region *split.RegionInfo) (*sst.IngestResponse, error) { - leader := region.Leader - if leader == nil { - leader = region.Region.GetPeers()[0] - } - - cli, err := local.getImportClient(ctx, leader.StoreId) - if err != nil { - return nil, err - } - reqCtx := &kvrpcpb.Context{ - RegionId: region.Region.GetId(), - RegionEpoch: region.Region.GetRegionEpoch(), - Peer: leader, - } - - if !local.supportMultiIngest { - if len(metas) != 1 { - return nil, errors.New("batch ingest is not support") - } - req := &sst.IngestRequest{ - Context: reqCtx, - Sst: metas[0], - } - resp, err := cli.Ingest(ctx, req) - return resp, errors.Trace(err) - } - - if local.shouldCheckWriteStall { - writeStall, resp, err := local.checkWriteStall(ctx, region) - if err != nil { - return nil, errors.Trace(err) - } - if writeStall { - return resp, nil - } - } - - req := &sst.MultiIngestRequest{ - Context: reqCtx, - Ssts: metas, - } - resp, err := cli.MultiIngest(ctx, req) - return resp, errors.Trace(err) -} - -func (local *local) checkWriteStall(ctx context.Context, region *split.RegionInfo) (bool, *sst.IngestResponse, error) { - for _, peer := range region.Region.GetPeers() { - cli, err := local.getImportClient(ctx, peer.StoreId) - if err != nil { - return false, nil, errors.Trace(err) - } - // currently we use empty MultiIngestRequest to check if TiKV is busy. - // If in future the rate limit feature contains more metrics we can switch to use it. - resp, err := cli.MultiIngest(ctx, &sst.MultiIngestRequest{}) - if err != nil { - return false, nil, errors.Trace(err) - } - if resp.Error != nil && resp.Error.ServerIsBusy != nil { - return true, resp, nil - } - } - return false, nil, nil -} - func splitRangeBySizeProps(fullRange Range, sizeProps *sizeProperties, sizeLimit int64, keysLimit int64) []Range { ranges := make([]Range, 0, sizeProps.totalSize/uint64(sizeLimit)) curSize := uint64(0) @@ -1327,86 +1018,155 @@ func (local *local) readAndSplitIntoRange(ctx context.Context, engine *Engine, r return ranges, nil } -func (local *local) writeAndIngestByRange( +// prepareAndGenerateUnfinishedJob will read the engine to get unfinished key range, +// then split and scatter regions for these range and generate region jobs. +func (local *local) prepareAndGenerateUnfinishedJob( ctx context.Context, - engine *Engine, - start, end []byte, - regionSplitSize int64, - regionSplitKeys int64, -) error { - pairStart, pairEnd, err := engine.getFirstAndLastKey(start, end) - if err != nil { - return err - } - if pairStart == nil { - log.FromContext(ctx).Info("There is no pairs in iterator", - logutil.Key("start", start), - logutil.Key("end", end)) - engine.finishedRanges.add(Range{start: start, end: end}) - return nil + engineUUID uuid.UUID, + lf *Engine, + initialSplitRanges []Range, + regionSplitSize, regionSplitKeys int64, +) ([]*regionJob, error) { + lfTotalSize := lf.TotalSize.Load() + lfLength := lf.Length.Load() + unfinishedRanges := lf.unfinishedRanges(initialSplitRanges) + log.FromContext(ctx).Info("import engine unfinished ranges", zap.Int("count", len(unfinishedRanges))) + + if len(unfinishedRanges) > 0 { + // if all the kv can fit in one region, skip split regions. TiDB will split one region for + // the table when table is created. + needSplit := len(unfinishedRanges) > 1 || lfTotalSize > regionSplitSize || lfLength > regionSplitKeys + var err error + // split region by given ranges + failpoint.Inject("failToSplit", func(_ failpoint.Value) { + needSplit = true + }) + for i := 0; i < maxRetryTimes; i++ { + err = local.SplitAndScatterRegionInBatches(ctx, unfinishedRanges, lf.tableInfo, needSplit, regionSplitSize, maxBatchSplitRanges) + if err == nil || common.IsContextCanceledError(err) { + break + } + + log.FromContext(ctx).Warn("split and scatter failed in retry", zap.Stringer("uuid", engineUUID), + log.ShortError(err), zap.Int("retry", i)) + } + if err != nil { + log.FromContext(ctx).Error("split & scatter ranges failed", zap.Stringer("uuid", engineUUID), log.ShortError(err)) + return nil, err + } } - var regions []*split.RegionInfo + return local.generateJobInRanges( + ctx, + lf, + unfinishedRanges, + regionSplitSize, + regionSplitKeys, + ) +} -ScanWriteIngest: - for retry := 0; retry < maxRetryTimes; { - if retry != 0 { - select { - case <-time.After(time.Second): - case <-ctx.Done(): - return ctx.Err() - } +// generateJobInRanges scans the region in ranges and generate region jobs. +func (local *local) generateJobInRanges( + ctx context.Context, + engine *Engine, + ranges []Range, + regionSplitSize, regionSplitKeys int64, +) ([]*regionJob, error) { + log.FromContext(ctx).Debug("the ranges Length write to tikv", zap.Int("Length", len(ranges))) + + ret := make([]*regionJob, 0, len(ranges)) + + for _, r := range ranges { + start, end := r.start, r.end + pairStart, pairEnd, err := engine.getFirstAndLastKey(start, end) + if err != nil { + return nil, err + } + if pairStart == nil { + log.FromContext(ctx).Info("There is no pairs in range", + logutil.Key("start", start), + logutil.Key("end", end)) + engine.finishedRanges.add(Range{start: start, end: end}) + continue } + startKey := codec.EncodeBytes([]byte{}, pairStart) endKey := codec.EncodeBytes([]byte{}, nextKey(pairEnd)) - regions, err = split.PaginateScanRegion(ctx, local.splitCli, startKey, endKey, scanRegionLimit) - if err != nil || len(regions) == 0 { - log.FromContext(ctx).Warn("scan region failed", log.ShortError(err), zap.Int("region_len", len(regions)), - logutil.Key("startKey", startKey), logutil.Key("endKey", endKey), zap.Int("retry", retry)) - retry++ - continue ScanWriteIngest + regions, err := split.PaginateScanRegion(ctx, local.splitCli, startKey, endKey, scanRegionLimit) + if err != nil { + log.FromContext(ctx).Error("scan region failed", + log.ShortError(err), zap.Int("region_len", len(regions)), + logutil.Key("startKey", startKey), + logutil.Key("endKey", endKey)) + return nil, err } for _, region := range regions { - log.FromContext(ctx).Debug("get region", zap.Int("retry", retry), zap.Binary("startKey", startKey), - zap.Binary("endKey", endKey), zap.Uint64("id", region.Region.GetId()), - zap.Stringer("epoch", region.Region.GetRegionEpoch()), zap.Binary("start", region.Region.GetStartKey()), - zap.Binary("end", region.Region.GetEndKey()), zap.Reflect("peers", region.Region.GetPeers())) - - w := local.ingestConcurrency.Apply() - err = local.writeAndIngestPairs(ctx, engine, region, pairStart, end, regionSplitSize, regionSplitKeys) - local.ingestConcurrency.Recycle(w) - if err != nil { - if !local.isRetryableImportTiKVError(err) { - return err - } - _, regionStart, _ := codec.DecodeBytes(region.Region.StartKey, []byte{}) - // if we have at least succeeded one region, retry without increasing the retry count - if bytes.Compare(regionStart, pairStart) > 0 { - pairStart = regionStart - } else { - retry++ - } - log.FromContext(ctx).Info("retry write and ingest kv pairs", logutil.Key("startKey", pairStart), - logutil.Key("endKey", end), log.ShortError(err), zap.Int("retry", retry)) - continue ScanWriteIngest + log.FromContext(ctx).Debug("get region", + zap.Binary("startKey", startKey), + zap.Binary("endKey", endKey), + zap.Uint64("id", region.Region.GetId()), + zap.Stringer("epoch", region.Region.GetRegionEpoch()), + zap.Binary("start", region.Region.GetStartKey()), + zap.Binary("end", region.Region.GetEndKey()), + zap.Reflect("peers", region.Region.GetPeers())) + + job := ®ionJob{ + keyRange: intersectRange(region.Region, Range{start: start, end: end}), + region: region, + stage: regionScanned, + engine: engine, + regionSplitSize: regionSplitSize, + regionSplitKeys: regionSplitKeys, + metrics: local.metrics, } - } - return err + ret = append(ret, job) + } } - - return err + return ret, nil } -type retryType int +// startWorker creates a worker that reads from the job channel and processes. +// startWorker will return nil if it's expected to stop, which means the context +// is canceled or channel is closed. It will return not nil error when it actively +// stops. +// this function must send the job back to jobOutCh after read it from jobInCh, +// even if any error happens. +func (local *local) startWorker( + ctx context.Context, + jobInCh, jobOutCh chan *regionJob, +) error { + for { + select { + case <-ctx.Done(): + return nil + case job, ok := <-jobInCh: + if !ok { + return nil + } -const ( - retryNone retryType = iota - retryWrite - retryIngest - retryBusyIngest -) + now := time.Now() + if now.Before(job.waitUntil) { + duration := job.waitUntil.Sub(now) + log.FromContext(ctx).Debug("need to wait before processing this job", + zap.Duration("wait", duration)) + select { + case <-ctx.Done(): + jobOutCh <- job + return nil + case <-time.After(duration): + } + } + + err := local.executeJob(ctx, job) + jobOutCh <- job + if err != nil { + return err + } + } + } +} func (local *local) isRetryableImportTiKVError(err error) bool { err = errors.Cause(err) @@ -1421,207 +1181,78 @@ func (local *local) isRetryableImportTiKVError(err error) bool { return common.IsRetryableError(err) } -// writeAndIngestPairs writes the kv pairs in the range [start, end) to the peers -// of the region, and then send the ingest command to do RocksDB ingest. -// when return nil, it does not mean the whole task success. The success ranges is -// recorded in the engine.finishedRanges. -// TODO: regionSplitSize and regionSplitKeys can be a member of Engine, no need to pass it in every function. -func (local *local) writeAndIngestPairs( +// executeJob handles a regionJob and tries to convert it to ingested stage. +// The ingested job will record finished ranges in engine as a checkpoint. +// If non-retryable error occurs, it will return the error. +// If retryable error occurs, it will return nil and caller should check the stage +// of the regionJob to determine what to do with it. +func (local *local) executeJob( ctx context.Context, - engine *Engine, - region *split.RegionInfo, - start, end []byte, - regionSplitSize int64, - regionSplitKeys int64, + job *regionJob, ) error { - var err error - var writeResult *tikvWriteResult -loopWrite: - for i := 0; i < maxRetryTimes; i++ { - writeResult, err = local.WriteToTiKV(ctx, engine, region, start, end, regionSplitSize, regionSplitKeys) - if err != nil { - if !local.isRetryableImportTiKVError(err) { - return err - } - - log.FromContext(ctx).Warn("write to tikv failed", log.ShortError(err), zap.Int("retry", i)) - continue loopWrite - } - metas, finishedRange, rangeStats := writeResult.sstMeta, writeResult.finishedRange, writeResult.rangeStats - - if len(metas) == 0 { - return nil - } - - batch := 1 - if local.supportMultiIngest { - batch = len(metas) - } - - for i := 0; i < len(metas); i += batch { - start := i * batch - end := mathutil.Min((i+1)*batch, len(metas)) - ingestMetas := metas[start:end] - errCnt := 0 - for errCnt < maxRetryTimes { - log.FromContext(ctx).Debug("ingest meta", zap.Reflect("meta", ingestMetas)) - var resp *sst.IngestResponse - failpoint.Inject("FailIngestMeta", func(val failpoint.Value) { - // only inject the error once - switch val.(string) { - case "notleader": - resp = &sst.IngestResponse{ - Error: &errorpb.Error{ - NotLeader: &errorpb.NotLeader{ - RegionId: region.Region.Id, - Leader: region.Leader, - }, - }, - } - case "epochnotmatch": - resp = &sst.IngestResponse{ - Error: &errorpb.Error{ - EpochNotMatch: &errorpb.EpochNotMatch{ - CurrentRegions: []*metapb.Region{region.Region}, - }, - }, - } - } - if resp != nil { - err = nil - } - }) - if resp == nil { - resp, err = local.Ingest(ctx, ingestMetas, region) - } + failpoint.Inject("WriteToTiKVNotEnoughDiskSpace", func(_ failpoint.Value) { + failpoint.Return( + errors.Errorf("The available disk of TiKV (%s) only left %d, and capacity is %d", "", 0, 0)) + }) + if local.shouldCheckTiKV { + for _, peer := range job.region.Region.GetPeers() { + var ( + store *pdtypes.StoreInfo + err error + ) + for i := 0; i < maxRetryTimes; i++ { + store, err = local.pdCtl.GetStoreInfo(ctx, peer.StoreId) if err != nil { - if common.IsContextCanceledError(err) { - return err - } - log.FromContext(ctx).Warn("ingest failed", log.ShortError(err), logutil.SSTMetas(ingestMetas), - logutil.Region(region.Region), logutil.Leader(region.Leader)) - errCnt++ continue } - - var retryTy retryType - var newRegion *split.RegionInfo - retryTy, newRegion, err = local.isIngestRetryable(ctx, resp, region, ingestMetas) - if common.IsContextCanceledError(err) { - return err - } - if err == nil { - // ingest next meta - break - } - - switch retryTy { - case retryNone: - log.FromContext(ctx).Warn("ingest failed noretry", log.ShortError(err), logutil.SSTMetas(ingestMetas), - logutil.Region(region.Region), logutil.Leader(region.Leader)) - // met non-retryable error retry whole Write procedure - return err - case retryWrite: - region = newRegion - continue loopWrite - case retryIngest: - region = newRegion - continue - case retryBusyIngest: - log.FromContext(ctx).Warn("meet tikv busy when ingest", log.ShortError(err), logutil.SSTMetas(ingestMetas), - logutil.Region(region.Region)) - // ImportEngine will continue on this unfinished range - return nil + if store.Status.Capacity > 0 { + // The available disk percent of TiKV + ratio := store.Status.Available * 100 / store.Status.Capacity + if ratio < 10 { + return errors.Errorf("The available disk of TiKV (%s) only left %d, and capacity is %d", + store.Store.Address, store.Status.Available, store.Status.Capacity) + } } + break } - } - - if err == nil { - engine.importedKVSize.Add(rangeStats.totalBytes) - engine.importedKVCount.Add(rangeStats.count) - engine.finishedRanges.add(finishedRange) - if local.metrics != nil { - local.metrics.BytesCounter.WithLabelValues(metric.BytesStateImported).Add(float64(rangeStats.totalBytes)) + if err != nil { + log.FromContext(ctx).Error("failed to get StoreInfo from pd http api", zap.Error(err)) } - return nil } - - log.FromContext(ctx).Warn("write and ingest region, will retry import full range", log.ShortError(err), - logutil.Region(region.Region), logutil.Key("start", start), - logutil.Key("end", end)) - return errors.Trace(err) } - return errors.Trace(err) -} -func (local *local) writeAndIngestByRanges(ctx context.Context, engine *Engine, ranges []Range, regionSplitSize int64, regionSplitKeys int64) error { - if engine.Length.Load() == 0 { - // engine is empty, this is likes because it's a index engine but the table contains no index - log.FromContext(ctx).Info("engine contains no data", zap.Stringer("uuid", engine.UUID)) + err := job.writeToTiKV(ctx, + local.tikvCodec.GetAPIVersion(), + local.importClientFactory, + local.kvWriteBatchSize, + local.bufferPool, + local.writeLimiter) + if err != nil { + if !local.isRetryableImportTiKVError(err) { + return err + } + log.FromContext(ctx).Warn("meet retryable error when writing to TiKV", + log.ShortError(err), zap.Stringer("job stage", job.stage)) + job.lastRetryableErr = err return nil } - log.FromContext(ctx).Debug("the ranges Length write to tikv", zap.Int("Length", len(ranges))) - - var allErrLock sync.Mutex - var allErr error - var wg sync.WaitGroup - metErr := atomic.NewBool(false) - for _, r := range ranges { - startKey := r.start - endKey := r.end - w := local.rangeConcurrency.Apply() - // if meet error here, skip try more here to allow fail fast. - if metErr.Load() { - local.rangeConcurrency.Recycle(w) - break + err = job.ingest( + ctx, + local.importClientFactory, + local.splitCli, + local.supportMultiIngest, + local.shouldCheckWriteStall, + ) + if err != nil { + if !local.isRetryableImportTiKVError(err) { + return err } - wg.Add(1) - go func(w *worker.Worker) { - defer func() { - local.rangeConcurrency.Recycle(w) - wg.Done() - }() - var err error - // max retry backoff time: 2+4+8+16+30*26=810s - backOffTime := time.Second - for i := 0; i < maxWriteAndIngestRetryTimes; i++ { - err = local.writeAndIngestByRange(ctx, engine, startKey, endKey, regionSplitSize, regionSplitKeys) - if err == nil || common.IsContextCanceledError(err) { - return - } - if !local.isRetryableImportTiKVError(err) { - break - } - log.FromContext(ctx).Warn("write and ingest by range failed", - zap.Int("retry time", i+1), log.ShortError(err)) - backOffTime *= 2 - if backOffTime > maxRetryBackoffTime { - backOffTime = maxRetryBackoffTime - } - select { - case <-time.After(backOffTime): - case <-ctx.Done(): - return - } - } - - allErrLock.Lock() - allErr = multierr.Append(allErr, err) - allErrLock.Unlock() - if err != nil { - metErr.Store(true) - } - }(w) + log.FromContext(ctx).Warn("meet retryable error when ingesting", + log.ShortError(err), zap.Stringer("job stage", job.stage)) + job.lastRetryableErr = err } - - // wait for all sub tasks finish to avoid panic. if we return on the first error, - // the outer tasks may close the pebble db but some sub tasks still read from the db - wg.Wait() - if allErr == nil { - return ctx.Err() - } - return allErr + return nil } func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regionSplitSize, regionSplitKeys int64) error { @@ -1635,6 +1266,7 @@ func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regi lfTotalSize := lf.TotalSize.Load() lfLength := lf.Length.Load() if lfTotalSize == 0 { + // engine is empty, this is likes because it's a index engine but the table contains no index log.FromContext(ctx).Info("engine contains no kv, skip import", zap.Stringer("engine", engineUUID)) return nil } @@ -1682,46 +1314,126 @@ func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regi failpoint.Inject("ReadyForImportEngine", func() {}) - for { - unfinishedRanges := lf.unfinishedRanges(ranges) - if len(unfinishedRanges) == 0 { - break + var ( + ctx2, workerCancel = context.WithCancel(ctx) + workGroup, workerCtx = errgroup.WithContext(ctx2) + // jobToWorkerCh is unbuffered so when we finished sending all jobs, we can make sure all jobs have been + // received by workers. + jobToWorkerCh = make(chan *regionJob) + jobFromWorkerCh = make(chan *regionJob) + // jobWg is the number of jobs that need to be processed by worker in this round. + jobWg sync.WaitGroup + jobsNeedRetry []*regionJob + retryErr atomic.Error + retryGoroutineDone = make(chan struct{}) + ) + + // handle processed job from worker, it will only exit when jobFromWorkerCh + // is closed to avoid send to jobFromWorkerCh is blocked. + defer func() { + close(jobFromWorkerCh) + <-retryGoroutineDone + }() + go func() { + defer close(retryGoroutineDone) + for { + job, ok := <-jobFromWorkerCh + if !ok { + return + } + switch job.stage { + case regionScanned, wrote: + job.retryCount++ + if job.retryCount > maxWriteAndIngestRetryTimes { + retryErr.Store(job.lastRetryableErr) + workerCancel() + jobWg.Done() + continue + } + // max retry backoff time: 2+4+8+16+30*26=810s + sleepSecond := math.Pow(2, float64(job.retryCount)) + if sleepSecond > maxRetryBackoffSecond { + sleepSecond = maxRetryBackoffSecond + } + job.waitUntil = time.Now().Add(time.Second * time.Duration(sleepSecond)) + log.FromContext(ctx).Info("put job back to jobCh to retry later", + logutil.Key("startKey", job.keyRange.start), + logutil.Key("endKey", job.keyRange.end), + zap.Stringer("stage", job.stage), + zap.Int("retryCount", job.retryCount), + zap.Time("waitUntil", job.waitUntil)) + jobsNeedRetry = append(jobsNeedRetry, job) + case ingested, needRescan: + } + jobWg.Done() } - log.FromContext(ctx).Info("import engine unfinished ranges", zap.Int("count", len(unfinishedRanges))) + }() - // if all the kv can fit in one region, skip split regions. TiDB will split one region for - // the table when table is created. - needSplit := len(unfinishedRanges) > 1 || lfTotalSize > regionSplitSize || lfLength > regionSplitKeys - // split region by given ranges - failpoint.Inject("failToSplit", func(_ failpoint.Value) { - needSplit = true + for i := 0; i < local.workerConcurrency; i++ { + workGroup.Go(func() error { + return local.startWorker(workerCtx, jobToWorkerCh, jobFromWorkerCh) }) - for i := 0; i < maxRetryTimes; i++ { - err = local.SplitAndScatterRegionInBatches(ctx, unfinishedRanges, lf.tableInfo, needSplit, regionSplitSize, maxBatchSplitRanges) - if err == nil || common.IsContextCanceledError(err) { - break - } + } - log.FromContext(ctx).Warn("split and scatter failed in retry", zap.Stringer("uuid", engineUUID), - log.ShortError(err), zap.Int("retry", i)) - } + var pendingJobs []*regionJob + for { + pendingJobs = append(pendingJobs, jobsNeedRetry...) + jobsNeedRetry = nil + log.FromContext(ctx).Info("import engine pending jobs", zap.Int("count", len(pendingJobs))) + + newJobs, err := local.prepareAndGenerateUnfinishedJob( + ctx, + engineUUID, + lf, + ranges, + regionSplitSize, + regionSplitKeys, + ) if err != nil { - log.FromContext(ctx).Error("split & scatter ranges failed", zap.Stringer("uuid", engineUUID), log.ShortError(err)) + close(jobToWorkerCh) + _ = workGroup.Wait() return err } - // start to write to kv and ingest - err = local.writeAndIngestByRanges(ctx, lf, unfinishedRanges, regionSplitSize, regionSplitKeys) - if err != nil { - log.FromContext(ctx).Error("write and ingest engine failed", log.ShortError(err)) - return err + pendingJobs = append(pendingJobs, newJobs...) + if len(pendingJobs) == 0 { + break + } + + jobWg.Add(len(pendingJobs)) + for _, job := range pendingJobs { + select { + case <-workerCtx.Done(): + err2 := retryErr.Load() + if err2 != nil { + return errors.Trace(err2) + } + return errors.Trace(workGroup.Wait()) + case jobToWorkerCh <- job: + } + } + pendingJobs = nil + // all jobs are received by workers and worker will always call jobWg.Done() after processing + // no need to check workerCtx.Done() here. + jobWg.Wait() + // check if worker has error in this round + select { + case <-workerCtx.Done(): + groupErr := workGroup.Wait() + err2 := retryErr.Load() + if err2 != nil { + return errors.Trace(err2) + } + return errors.Trace(groupErr) + default: } } + close(jobToWorkerCh) log.FromContext(ctx).Info("import engine success", zap.Stringer("uuid", engineUUID), zap.Int64("size", lfTotalSize), zap.Int64("kvs", lfLength), zap.Int64("importedSize", lf.importedKVSize.Load()), zap.Int64("importedCount", lf.importedKVCount.Load())) - return nil + return workGroup.Wait() } func (local *local) CollectLocalDuplicateRows(ctx context.Context, tbl table.Table, tableName string, opts *kv.SessionOptions) (hasDupe bool, err error) { @@ -1984,116 +1696,6 @@ func openLocalWriter(cfg *backend.LocalWriterConfig, engine *Engine, tikvCodec t return w, nil } -func (local *local) isIngestRetryable( - ctx context.Context, - resp *sst.IngestResponse, - region *split.RegionInfo, - metas []*sst.SSTMeta, -) (retryType, *split.RegionInfo, error) { - if resp.GetError() == nil { - return retryNone, nil, nil - } - - getRegion := func() (*split.RegionInfo, error) { - for i := 0; ; i++ { - newRegion, err := local.splitCli.GetRegion(ctx, region.Region.GetStartKey()) - if err != nil { - return nil, errors.Trace(err) - } - if newRegion != nil { - return newRegion, nil - } - log.FromContext(ctx).Warn("get region by key return nil, will retry", logutil.Region(region.Region), logutil.Leader(region.Leader), - zap.Int("retry", i)) - select { - case <-ctx.Done(): - return nil, ctx.Err() - case <-time.After(time.Second): - } - } - } - - var newRegion *split.RegionInfo - var err error - switch errPb := resp.GetError(); { - case errPb.NotLeader != nil: - if newLeader := errPb.GetNotLeader().GetLeader(); newLeader != nil { - newRegion = &split.RegionInfo{ - Leader: newLeader, - Region: region.Region, - } - } else { - newRegion, err = getRegion() - if err != nil { - return retryNone, nil, errors.Trace(err) - } - } - // TODO: because in some case, TiKV may return retryable error while the ingest is succeeded. - // Thus directly retry ingest may cause TiKV panic. So always return retryWrite here to avoid - // this issue. - // See: https://github.com/tikv/tikv/issues/9496 - return retryWrite, newRegion, common.ErrKVNotLeader.GenWithStack(errPb.GetMessage()) - case errPb.EpochNotMatch != nil: - if currentRegions := errPb.GetEpochNotMatch().GetCurrentRegions(); currentRegions != nil { - var currentRegion *metapb.Region - for _, r := range currentRegions { - if insideRegion(r, metas) { - currentRegion = r - break - } - } - if currentRegion != nil { - var newLeader *metapb.Peer - for _, p := range currentRegion.Peers { - if p.GetStoreId() == region.Leader.GetStoreId() { - newLeader = p - break - } - } - if newLeader != nil { - newRegion = &split.RegionInfo{ - Leader: newLeader, - Region: currentRegion, - } - } - } - } - retryTy := retryNone - if newRegion != nil { - retryTy = retryWrite - } - return retryTy, newRegion, common.ErrKVEpochNotMatch.GenWithStack(errPb.GetMessage()) - case strings.Contains(errPb.Message, "raft: proposal dropped"): - newRegion, err = getRegion() - if err != nil { - return retryNone, nil, errors.Trace(err) - } - return retryWrite, newRegion, common.ErrKVRaftProposalDropped.GenWithStack(errPb.GetMessage()) - case errPb.ServerIsBusy != nil: - return retryBusyIngest, nil, common.ErrKVServerIsBusy.GenWithStack(errPb.GetMessage()) - case errPb.RegionNotFound != nil: - return retryNone, nil, common.ErrKVRegionNotFound.GenWithStack(errPb.GetMessage()) - case errPb.ReadIndexNotReady != nil: - // this error happens when this region is splitting, the error might be: - // read index not ready, reason can not read index due to split, region 64037 - // we have paused schedule, but it's temporary, - // if next request takes a long time, there's chance schedule is enabled again - // or on key range border, another engine sharing this region tries to split this - // region may cause this error too. - newRegion, err = getRegion() - if err != nil { - return retryNone, nil, errors.Trace(err) - } - return retryWrite, newRegion, common.ErrKVReadIndexNotReady.GenWithStack(errPb.GetMessage()) - case errPb.DiskFull != nil: - return retryNone, nil, errors.Errorf("non-retryable error: %s", resp.GetError().GetMessage()) - } - // all others ingest error, such as stale command, etc. we'll retry it again from writeAndIngestByRange - // here we use a single named-error ErrKVIngestFailed to represent them all - // we can separate them later if it's needed - return retryNone, nil, common.ErrKVIngestFailed.GenWithStack(resp.GetError().GetMessage()) -} - // return the smallest []byte that is bigger than current bytes. // special case when key is empty, empty bytes means infinity in our context, so directly return itself. func nextKey(key []byte) []byte { diff --git a/br/pkg/lightning/backend/local/local_test.go b/br/pkg/lightning/backend/local/local_test.go index 0294cd7c5f05e..4a20d7adc5cc5 100644 --- a/br/pkg/lightning/backend/local/local_test.go +++ b/br/pkg/lightning/backend/local/local_test.go @@ -28,6 +28,7 @@ import ( "sync" "sync/atomic" "testing" + "time" "github.com/cockroachdb/pebble" "github.com/docker/go-units" @@ -37,12 +38,10 @@ import ( "github.com/pingcap/kvproto/pkg/errorpb" sst "github.com/pingcap/kvproto/pkg/import_sstpb" "github.com/pingcap/kvproto/pkg/metapb" - berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/log" - "github.com/pingcap/tidb/br/pkg/lightning/worker" "github.com/pingcap/tidb/br/pkg/membuf" "github.com/pingcap/tidb/br/pkg/pdutil" "github.com/pingcap/tidb/br/pkg/restore/split" @@ -440,108 +439,6 @@ func (c *mockSplitClient) GetRegion(ctx context.Context, key []byte) (*split.Reg }, nil } -func TestIsIngestRetryable(t *testing.T) { - local := &local{ - splitCli: &mockSplitClient{}, - logger: log.L(), - } - - resp := &sst.IngestResponse{ - Error: &errorpb.Error{ - NotLeader: &errorpb.NotLeader{ - Leader: &metapb.Peer{Id: 2}, - }, - }, - } - ctx := context.Background() - region := &split.RegionInfo{ - Leader: &metapb.Peer{Id: 1}, - Region: &metapb.Region{ - Id: 1, - StartKey: []byte{1}, - EndKey: []byte{3}, - RegionEpoch: &metapb.RegionEpoch{ - ConfVer: 1, - Version: 1, - }, - }, - } - metas := []*sst.SSTMeta{ - { - Range: &sst.Range{ - Start: []byte{1}, - End: []byte{2}, - }, - }, - { - Range: &sst.Range{ - Start: []byte{1, 1}, - End: []byte{2}, - }, - }, - } - retryType, newRegion, err := local.isIngestRetryable(ctx, resp, region, metas) - require.Equal(t, retryWrite, retryType) - require.Equal(t, uint64(2), newRegion.Leader.Id) - require.Error(t, err) - - resp.Error = &errorpb.Error{ - EpochNotMatch: &errorpb.EpochNotMatch{ - CurrentRegions: []*metapb.Region{ - { - Id: 1, - StartKey: []byte{1}, - EndKey: []byte{3}, - RegionEpoch: &metapb.RegionEpoch{ - ConfVer: 1, - Version: 2, - }, - Peers: []*metapb.Peer{{Id: 1}}, - }, - }, - }, - } - retryType, newRegion, err = local.isIngestRetryable(ctx, resp, region, metas) - require.Equal(t, retryWrite, retryType) - require.Equal(t, uint64(2), newRegion.Region.RegionEpoch.Version) - require.Error(t, err) - - resp.Error = &errorpb.Error{Message: "raft: proposal dropped"} - retryType, _, err = local.isIngestRetryable(ctx, resp, region, metas) - require.Equal(t, retryWrite, retryType) - require.Error(t, err) - - resp.Error = &errorpb.Error{ - ReadIndexNotReady: &errorpb.ReadIndexNotReady{ - Reason: "test", - }, - } - retryType, _, err = local.isIngestRetryable(ctx, resp, region, metas) - require.Equal(t, retryWrite, retryType) - require.Error(t, err) - - resp.Error = &errorpb.Error{ - Message: "raft: proposal dropped", - } - retryType, _, err = local.isIngestRetryable(ctx, resp, region, metas) - require.Equal(t, retryWrite, retryType) - require.True(t, berrors.Is(err, common.ErrKVRaftProposalDropped)) - - resp.Error = &errorpb.Error{ - DiskFull: &errorpb.DiskFull{}, - } - retryType, _, err = local.isIngestRetryable(ctx, resp, region, metas) - require.Equal(t, retryNone, retryType) - require.Contains(t, err.Error(), "non-retryable error") - - resp.Error = &errorpb.Error{ - StaleCommand: &errorpb.StaleCommand{}, - } - retryType, _, err = local.isIngestRetryable(ctx, resp, region, metas) - require.Equal(t, retryNone, retryType) - require.True(t, berrors.Is(err, common.ErrKVIngestFailed)) -} - type testIngester struct{} func (i testIngester) mergeSSTs(metas []*sstMeta, dir string) (*sstMeta, error) { @@ -1213,9 +1110,13 @@ func TestLocalWriteAndIngestPairsFailFast(t *testing.T) { defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/backend/local/WriteToTiKVNotEnoughDiskSpace")) }() - err := bak.writeAndIngestPairs(context.Background(), nil, nil, nil, nil, 0, 0) + jobCh := make(chan *regionJob, 1) + jobCh <- ®ionJob{} + jobOutCh := make(chan *regionJob, 1) + err := bak.startWorker(context.Background(), jobCh, jobOutCh) require.Error(t, err) require.Regexp(t, "The available disk of TiKV.*", err.Error()) + require.Len(t, jobCh, 0) } func TestGetRegionSplitSizeKeys(t *testing.T) { @@ -1258,13 +1159,9 @@ func TestLocalIsRetryableTiKVWriteError(t *testing.T) { } func TestCheckPeersBusy(t *testing.T) { - ctx := context.Background() - pdCli := &mockPdClient{} - pdCtl := &pdutil.PdController{} - pdCtl.SetPDClient(pdCli) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() - keys := [][]byte{[]byte(""), []byte("a"), []byte("b"), []byte("")} - splitCli := initTestSplitClient3Replica(keys, nil) apiInvokeRecorder := map[string][]uint64{} serverIsBusyResp := &sst.IngestResponse{ Error: &errorpb.Error{ @@ -1273,11 +1170,8 @@ func TestCheckPeersBusy(t *testing.T) { createTimeStore12 := 0 local := &local{ - pdCtl: pdCtl, - splitCli: splitCli, importClientFactory: &mockImportClientFactory{ stores: []*metapb.Store{ - // region ["", "a") is not used, skip (1, 2, 3) {Id: 11}, {Id: 12}, {Id: 13}, // region ["a", "b") {Id: 21}, {Id: 22}, {Id: 23}, // region ["b", "") }, @@ -1287,7 +1181,7 @@ func TestCheckPeersBusy(t *testing.T) { importCli.apiInvokeRecorder = apiInvokeRecorder if store.Id == 12 { createTimeStore12++ - // the second time to checkWriteStall + // the second time is checkWriteStall, we mock a busy response if createTimeStore12 == 2 { importCli.retry = 1 importCli.resp = serverIsBusyResp @@ -1297,7 +1191,6 @@ func TestCheckPeersBusy(t *testing.T) { }, }, logger: log.L(), - ingestConcurrency: worker.NewPool(ctx, 1, "ingest"), writeLimiter: noopStoreWriteLimiter{}, bufferPool: membuf.NewPool(), supportMultiIngest: true, @@ -1307,13 +1200,13 @@ func TestCheckPeersBusy(t *testing.T) { db, tmpPath := makePebbleDB(t, nil) _, engineUUID := backend.MakeUUID("ww", 0) - engineCtx, cancel := context.WithCancel(context.Background()) + engineCtx, cancel2 := context.WithCancel(context.Background()) f := &Engine{ db: db, UUID: engineUUID, sstDir: tmpPath, ctx: engineCtx, - cancel: cancel, + cancel: cancel2, sstMetasChan: make(chan metaOrFlush, 64), keyAdapter: noopKeyAdapter{}, logger: log.L(), @@ -1322,12 +1215,81 @@ func TestCheckPeersBusy(t *testing.T) { require.NoError(t, err) err = f.db.Set([]byte("b"), []byte("b"), nil) require.NoError(t, err) - err = local.writeAndIngestByRange(ctx, f, []byte("a"), []byte("c"), 0, 0) - require.NoError(t, err) + + jobCh := make(chan *regionJob, 10) + + retryJob := ®ionJob{ + keyRange: Range{start: []byte("a"), end: []byte("b")}, + region: &split.RegionInfo{ + Region: &metapb.Region{ + Id: 1, + Peers: []*metapb.Peer{ + {Id: 1, StoreId: 11}, {Id: 2, StoreId: 12}, {Id: 3, StoreId: 13}, + }, + StartKey: []byte("a"), + EndKey: []byte("b"), + }, + Leader: &metapb.Peer{Id: 1, StoreId: 11}, + }, + stage: regionScanned, + engine: f, + retryCount: 20, + waitUntil: time.Now().Add(-time.Second), + } + jobCh <- retryJob + + jobCh <- ®ionJob{ + keyRange: Range{start: []byte("b"), end: []byte("")}, + region: &split.RegionInfo{ + Region: &metapb.Region{ + Id: 4, + Peers: []*metapb.Peer{ + {Id: 4, StoreId: 21}, {Id: 5, StoreId: 22}, {Id: 6, StoreId: 23}, + }, + StartKey: []byte("b"), + EndKey: []byte(""), + }, + Leader: &metapb.Peer{Id: 4, StoreId: 21}, + }, + stage: regionScanned, + engine: f, + retryCount: 20, + waitUntil: time.Now().Add(-time.Second), + } + + retryJobs := make([]*regionJob, 0, 1) + + var wg sync.WaitGroup + wg.Add(1) + jobOutCh := make(chan *regionJob) + go func() { + job := <-jobOutCh + job.retryCount++ + retryJobs = append(retryJobs, job) + <-jobOutCh + wg.Done() + }() + wg.Add(1) + go func() { + defer wg.Done() + err := local.startWorker(ctx, jobCh, jobOutCh) + require.NoError(t, err) + }() + + // retryJob will be retried once and worker will sleep 30s before processing the + // job again, we simply hope below check is happened when worker is sleeping + time.Sleep(5 * time.Second) + require.Len(t, retryJobs, 1) + require.Same(t, retryJob, retryJobs[0]) + require.Equal(t, 21, retryJob.retryCount) + require.Equal(t, wrote, retryJob.stage) + + cancel() + wg.Wait() require.Equal(t, []uint64{11, 12, 13, 21, 22, 23}, apiInvokeRecorder["Write"]) // store 12 has a follower busy, so it will break the workflow for region (11, 12, 13) require.Equal(t, []uint64{11, 12, 21, 22, 23, 21}, apiInvokeRecorder["MultiIngest"]) // region (11, 12, 13) has key range ["a", "b"), it's not finished. - require.Equal(t, []Range{{start: []byte("b"), end: []byte("c")}}, f.finishedRanges.ranges) + require.Equal(t, []Range{{start: []byte("b"), end: []byte("")}}, f.finishedRanges.ranges) } diff --git a/br/pkg/lightning/backend/local/region_job.go b/br/pkg/lightning/backend/local/region_job.go new file mode 100644 index 0000000000000..16288eaa44c96 --- /dev/null +++ b/br/pkg/lightning/backend/local/region_job.go @@ -0,0 +1,628 @@ +// 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 local + +import ( + "context" + "strings" + "time" + + "github.com/cockroachdb/pebble" + "github.com/google/uuid" + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/errorpb" + sst "github.com/pingcap/kvproto/pkg/import_sstpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/br/pkg/lightning/common" + "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/pingcap/tidb/br/pkg/lightning/log" + "github.com/pingcap/tidb/br/pkg/lightning/metric" + "github.com/pingcap/tidb/br/pkg/logutil" + "github.com/pingcap/tidb/br/pkg/membuf" + "github.com/pingcap/tidb/br/pkg/restore/split" + "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/mathutil" + "go.uber.org/zap" +) + +type jobStageTp string + +// nil -> regionScanned: create a new region job +// regionScanned -> wrote: write the data to TiKV +// wrote -> ingested: ingest the data to TiKV +// ingested -> nil: finish the job +// regionScanned / wrote -> needRescan: need to rescan the data, maybe region is expanded. +// needRescan -> nil: discard the job. caller will create a new job from unfinishedRanges. +const ( + regionScanned jobStageTp = "regionScanned" + wrote jobStageTp = "wrote" + ingested jobStageTp = "ingested" + needRescan jobStageTp = "needRescan" +) + +func (j jobStageTp) String() string { + return string(j) +} + +// regionJob is dedicated to import the data in [keyRange.start, keyRange.end) to a region. +type regionJob struct { + keyRange Range + // TODO: check the keyRange so that it's always included in region + region *split.RegionInfo + // stage should be updated only by convertStageTo + stage jobStageTp + + engine *Engine + regionSplitSize int64 + regionSplitKeys int64 + metrics *metric.Metrics + // below fields are available after wrote stage + writeResult *tikvWriteResult + + retryCount int + waitUntil time.Time + lastRetryableErr error +} + +type tikvWriteResult struct { + sstMeta []*sst.SSTMeta + rangeStats rangeStats +} + +type rangeStats struct { + count int64 + totalBytes int64 +} + +func (j *regionJob) convertStageTo(stage jobStageTp) { + j.stage = stage + switch stage { + case regionScanned: + j.writeResult = nil + case ingested: + j.engine.finishedRanges.add(j.keyRange) + + // when writing is skipped because range is empty + if j.writeResult == nil { + return + } + + j.engine.importedKVSize.Add(j.writeResult.rangeStats.totalBytes) + j.engine.importedKVCount.Add(j.writeResult.rangeStats.count) + if j.metrics != nil { + j.metrics.BytesCounter.WithLabelValues(metric.BytesStateImported). + Add(float64(j.writeResult.rangeStats.totalBytes)) + } + } +} + +// writeToTiKV writes the data to TiKV and mark this job as wrote stage. +// if any write logic has error, writeToTiKV will set job to a proper stage and return nil. +// if any underlying logic has error, writeToTiKV will return an error. +// we don't need to do cleanup for the pairs written to tikv if encounters an error, +// tikv will take the responsibility to do so. +// TODO: let client-go provide a high-level write interface. +func (j *regionJob) writeToTiKV( + ctx context.Context, + apiVersion kvrpcpb.APIVersion, + clientFactory ImportClientFactory, + kvBatchSize int, + bufferPool *membuf.Pool, + writeLimiter StoreWriteLimiter, +) error { + if j.stage != regionScanned { + return nil + } + + begin := time.Now() + stats := rangeStats{} + region := j.region.Region + + firstKey, lastKey, err := j.engine.getFirstAndLastKey(j.keyRange.start, j.keyRange.end) + if err != nil { + return errors.Trace(err) + } + if firstKey == nil { + j.convertStageTo(ingested) + log.FromContext(ctx).Info("keys within region is empty, skip doIngest", + logutil.Key("start", j.keyRange.start), + logutil.Key("regionStart", region.StartKey), + logutil.Key("end", j.keyRange.end), + logutil.Key("regionEnd", region.EndKey)) + return nil + } + + firstKey = codec.EncodeBytes([]byte{}, firstKey) + lastKey = codec.EncodeBytes([]byte{}, lastKey) + + u := uuid.New() + meta := &sst.SSTMeta{ + Uuid: u[:], + RegionId: region.GetId(), + RegionEpoch: region.GetRegionEpoch(), + Range: &sst.Range{ + Start: firstKey, + End: lastKey, + }, + ApiVersion: apiVersion, + } + + leaderID := j.region.Leader.GetId() + clients := make([]sst.ImportSST_WriteClient, 0, len(region.GetPeers())) + storeIDs := make([]uint64, 0, len(region.GetPeers())) + requests := make([]*sst.WriteRequest, 0, len(region.GetPeers())) + for _, peer := range region.GetPeers() { + cli, err := clientFactory.Create(ctx, peer.StoreId) + if err != nil { + return errors.Trace(err) + } + + wstream, err := cli.Write(ctx) + if err != nil { + return errors.Trace(err) + } + + // Bind uuid for this write request + req := &sst.WriteRequest{ + Chunk: &sst.WriteRequest_Meta{ + Meta: meta, + }, + } + if err = wstream.Send(req); err != nil { + return errors.Trace(err) + } + req.Chunk = &sst.WriteRequest_Batch{ + Batch: &sst.WriteBatch{ + CommitTs: j.engine.TS, + }, + } + clients = append(clients, wstream) + requests = append(requests, req) + storeIDs = append(storeIDs, peer.StoreId) + } + + bytesBuf := bufferPool.NewBuffer() + defer bytesBuf.Destroy() + pairs := make([]*sst.Pair, 0, kvBatchSize) + count := 0 + size := int64(0) + totalSize := int64(0) + totalCount := int64(0) + // if region-split-size <= 96MiB, we bump the threshold a bit to avoid too many retry split + // because the range-properties is not 100% accurate + regionMaxSize := j.regionSplitSize + if j.regionSplitSize <= int64(config.SplitRegionSize) { + regionMaxSize = j.regionSplitSize * 4 / 3 + } + // Set a lower flush limit to make the speed of write more smooth. + flushLimit := int64(writeLimiter.Limit() / 10) + + flushKVs := func() error { + for i := range clients { + if err := writeLimiter.WaitN(ctx, storeIDs[i], int(size)); err != nil { + return errors.Trace(err) + } + // TODO: concurrent write? + requests[i].Chunk.(*sst.WriteRequest_Batch).Batch.Pairs = pairs[:count] + if err := clients[i].Send(requests[i]); err != nil { + return errors.Trace(err) + } + } + return nil + } + + opt := &pebble.IterOptions{LowerBound: j.keyRange.start, UpperBound: j.keyRange.end} + iter := j.engine.newKVIter(ctx, opt) + //nolint: errcheck + defer iter.Close() + + for iter.First(); iter.Valid(); iter.Next() { + kvSize := int64(len(iter.Key()) + len(iter.Value())) + // here we reuse the `*sst.Pair`s to optimize object allocation + if count < len(pairs) { + pairs[count].Key = bytesBuf.AddBytes(iter.Key()) + pairs[count].Value = bytesBuf.AddBytes(iter.Value()) + } else { + pair := &sst.Pair{ + Key: bytesBuf.AddBytes(iter.Key()), + Value: bytesBuf.AddBytes(iter.Value()), + } + pairs = append(pairs, pair) + } + count++ + totalCount++ + size += kvSize + totalSize += kvSize + + if count >= kvBatchSize || size >= flushLimit { + if err := flushKVs(); err != nil { + return errors.Trace(err) + } + count = 0 + size = 0 + bytesBuf.Reset() + } + if totalSize >= regionMaxSize || totalCount >= j.regionSplitKeys { + // we will shrink the key range of this job to real written range + if iter.Valid() && iter.Next() { + firstKey := append([]byte{}, iter.Key()...) + oldEndKey := j.keyRange.end + j.keyRange.end = firstKey + log.FromContext(ctx).Info("write to tikv partial finish", + zap.Int64("count", totalCount), + zap.Int64("size", totalSize), + logutil.Key("startKey", j.keyRange.start), + logutil.Key("endKey", oldEndKey), + logutil.Key("remainStart", firstKey), + logutil.Key("remainEnd", oldEndKey), + logutil.Region(region), + logutil.Leader(j.region.Leader)) + } + break + } + } + + if iter.Error() != nil { + return errors.Trace(iter.Error()) + } + + if count > 0 { + if err := flushKVs(); err != nil { + return errors.Trace(err) + } + count = 0 + size = 0 + bytesBuf.Reset() + } + + var leaderPeerMetas []*sst.SSTMeta + for i, wStream := range clients { + resp, closeErr := wStream.CloseAndRecv() + if closeErr != nil { + return errors.Trace(closeErr) + } + if resp.Error != nil { + return errors.New(resp.Error.Message) + } + if leaderID == region.Peers[i].GetId() { + leaderPeerMetas = resp.Metas + log.FromContext(ctx).Debug("get metas after write kv stream to tikv", zap.Reflect("metas", leaderPeerMetas)) + } + } + + // if there is not leader currently, we don't forward the stage to wrote and let caller + // handle the retry. + if len(leaderPeerMetas) == 0 { + log.FromContext(ctx).Warn("write to tikv no leader", + 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) + } + + log.FromContext(ctx).Debug("write to kv", zap.Reflect("region", j.region), zap.Uint64("leader", leaderID), + zap.Reflect("meta", meta), zap.Reflect("return metas", leaderPeerMetas), + zap.Int64("kv_pairs", totalCount), zap.Int64("total_bytes", totalSize), + zap.Int64("buf_size", bytesBuf.TotalSize()), + zap.Stringer("takeTime", time.Since(begin))) + + stats.count = totalCount + stats.totalBytes = totalSize + j.writeResult = &tikvWriteResult{ + sstMeta: leaderPeerMetas, + rangeStats: stats, + } + j.convertStageTo(wrote) + return nil +} + +// ingest tries to finish the regionJob. +// if any ingest logic has error, ingest may retry sometimes to resolve it and finally +// set job to a proper stage with nil error returned. +// if any underlying logic has error, ingest will return an error to let caller +// handle it. +func (j *regionJob) ingest( + ctx context.Context, + clientFactory ImportClientFactory, + splitCli split.SplitClient, + supportMultiIngest bool, + shouldCheckWriteStall bool, +) error { + switch j.stage { + case regionScanned, ingested: + return nil + case wrote: + } + + if len(j.writeResult.sstMeta) == 0 { + j.convertStageTo(ingested) + return nil + } + + for retry := 0; retry < maxRetryTimes; retry++ { + resp, err := j.doIngest(ctx, clientFactory, supportMultiIngest, shouldCheckWriteStall) + if err == nil && resp.GetError() == nil { + j.convertStageTo(ingested) + return nil + } + if err != nil { + if common.IsContextCanceledError(err) { + return err + } + log.FromContext(ctx).Warn("meet underlying error, will retry ingest", + log.ShortError(err), logutil.SSTMetas(j.writeResult.sstMeta), + logutil.Region(j.region.Region), logutil.Leader(j.region.Leader)) + continue + } + canContinue, err := j.fixIngestError(ctx, resp, splitCli) + if common.IsContextCanceledError(err) { + return err + } + if !canContinue { + log.FromContext(ctx).Warn("meet error and handle the job later", + zap.Stringer("job stage", j.stage), + logutil.ShortError(j.lastRetryableErr), + logutil.Region(j.region.Region), + logutil.Key("start", j.keyRange.start), + logutil.Key("end", j.keyRange.end)) + return nil + } + log.FromContext(ctx).Warn("meet error and will doIngest region, again", + logutil.ShortError(j.lastRetryableErr), + logutil.Region(j.region.Region), + logutil.Key("start", j.keyRange.start), + logutil.Key("end", j.keyRange.end)) + } + return nil +} + +func (j *regionJob) checkWriteStall( + ctx context.Context, + region *split.RegionInfo, + clientFactory ImportClientFactory, +) (bool, *sst.IngestResponse, error) { + for _, peer := range region.Region.GetPeers() { + cli, err := clientFactory.Create(ctx, peer.StoreId) + if err != nil { + return false, nil, errors.Trace(err) + } + // currently we use empty MultiIngestRequest to check if TiKV is busy. + // If in future the rate limit feature contains more metrics we can switch to use it. + resp, err := cli.MultiIngest(ctx, &sst.MultiIngestRequest{}) + if err != nil { + return false, nil, errors.Trace(err) + } + if resp.Error != nil && resp.Error.ServerIsBusy != nil { + return true, resp, nil + } + } + return false, nil, nil +} + +// doIngest send ingest commands to TiKV based on regionJob.writeResult.sstMeta. +// When meet error, it will remove finished sstMetas before return. +func (j *regionJob) doIngest( + ctx context.Context, + clientFactory ImportClientFactory, + supportMultiIngest bool, + shouldCheckWriteStall bool, +) (*sst.IngestResponse, error) { + if shouldCheckWriteStall { + writeStall, resp, err := j.checkWriteStall(ctx, j.region, clientFactory) + if err != nil { + return nil, errors.Trace(err) + } + if writeStall { + return resp, nil + } + } + + batch := 1 + if supportMultiIngest { + batch = len(j.writeResult.sstMeta) + } + + var resp *sst.IngestResponse + for start := 0; start < len(j.writeResult.sstMeta); start += batch { + end := mathutil.Min(start+batch, len(j.writeResult.sstMeta)) + ingestMetas := j.writeResult.sstMeta[start:end] + + log.FromContext(ctx).Debug("ingest meta", zap.Reflect("meta", ingestMetas)) + + failpoint.Inject("FailIngestMeta", func(val failpoint.Value) { + // only inject the error once + var resp *sst.IngestResponse + + switch val.(string) { + case "notleader": + resp = &sst.IngestResponse{ + Error: &errorpb.Error{ + NotLeader: &errorpb.NotLeader{ + RegionId: j.region.Region.Id, + Leader: j.region.Leader, + }, + }, + } + case "epochnotmatch": + resp = &sst.IngestResponse{ + Error: &errorpb.Error{ + EpochNotMatch: &errorpb.EpochNotMatch{ + CurrentRegions: []*metapb.Region{j.region.Region}, + }, + }, + } + } + failpoint.Return(resp, nil) + }) + + leader := j.region.Leader + if leader == nil { + leader = j.region.Region.GetPeers()[0] + } + + cli, err := clientFactory.Create(ctx, leader.StoreId) + if err != nil { + return nil, errors.Trace(err) + } + reqCtx := &kvrpcpb.Context{ + RegionId: j.region.Region.GetId(), + RegionEpoch: j.region.Region.GetRegionEpoch(), + Peer: leader, + } + + if supportMultiIngest { + req := &sst.MultiIngestRequest{ + Context: reqCtx, + Ssts: ingestMetas, + } + resp, err = cli.MultiIngest(ctx, req) + } else { + req := &sst.IngestRequest{ + Context: reqCtx, + Sst: ingestMetas[0], + } + resp, err = cli.Ingest(ctx, req) + } + if resp.GetError() != nil || err != nil { + // remove finished sstMetas + j.writeResult.sstMeta = j.writeResult.sstMeta[start:] + return resp, errors.Trace(err) + } + } + return resp, nil +} + +// fixIngestError will try to fix the error contained in ingest response. +// Return (_, error) when another error occurred. +// Return (true, nil) when the job can retry ingesting immediately. +// Return (false, nil) when the job should be put back to queue. +func (j *regionJob) fixIngestError( + ctx context.Context, + resp *sst.IngestResponse, + splitCli split.SplitClient, +) (bool, error) { + if resp.GetError() == nil { + return true, nil + } + + getRegion := func() (*split.RegionInfo, error) { + for i := 0; ; i++ { + newRegion, err := splitCli.GetRegion(ctx, j.region.Region.GetStartKey()) + if err != nil { + return nil, errors.Trace(err) + } + if newRegion != nil { + return newRegion, nil + } + log.FromContext(ctx).Warn("get region by key return nil, will retry", + logutil.Region(j.region.Region), logutil.Leader(j.region.Leader), + zap.Int("retry", i)) + select { + case <-ctx.Done(): + return nil, ctx.Err() + case <-time.After(time.Second): + } + } + } + + var newRegion *split.RegionInfo + var err error + switch errPb := resp.GetError(); { + case errPb.NotLeader != nil: + j.lastRetryableErr = common.ErrKVNotLeader.GenWithStack(errPb.GetMessage()) + + if newLeader := errPb.GetNotLeader().GetLeader(); newLeader != nil { + newRegion = &split.RegionInfo{ + Leader: newLeader, + Region: j.region.Region, + } + } else { + newRegion, err = getRegion() + if err != nil { + return false, errors.Trace(err) + } + } + j.region = newRegion + return true, nil + case errPb.EpochNotMatch != nil: + j.lastRetryableErr = common.ErrKVEpochNotMatch.GenWithStack(errPb.GetMessage()) + + if currentRegions := errPb.GetEpochNotMatch().GetCurrentRegions(); currentRegions != nil { + var currentRegion *metapb.Region + for _, r := range currentRegions { + if insideRegion(r, j.writeResult.sstMeta) { + currentRegion = r + break + } + } + if currentRegion != nil { + var newLeader *metapb.Peer + for _, p := range currentRegion.Peers { + if p.GetStoreId() == j.region.Leader.GetStoreId() { + newLeader = p + break + } + } + if newLeader != nil { + newRegion = &split.RegionInfo{ + Leader: newLeader, + Region: currentRegion, + } + } + } + } + if newRegion != nil { + j.region = newRegion + j.convertStageTo(regionScanned) + return false, nil + } + j.convertStageTo(needRescan) + return false, nil + case strings.Contains(errPb.Message, "raft: proposal dropped"): + j.lastRetryableErr = common.ErrKVRaftProposalDropped.GenWithStack(errPb.GetMessage()) + + j.convertStageTo(needRescan) + return false, nil + case errPb.ServerIsBusy != nil: + j.lastRetryableErr = common.ErrKVServerIsBusy.GenWithStack(errPb.GetMessage()) + + return false, nil + case errPb.RegionNotFound != nil: + j.lastRetryableErr = common.ErrKVRegionNotFound.GenWithStack(errPb.GetMessage()) + + j.convertStageTo(needRescan) + return false, nil + case errPb.ReadIndexNotReady != nil: + j.lastRetryableErr = common.ErrKVReadIndexNotReady.GenWithStack(errPb.GetMessage()) + + // this error happens when this region is splitting, the error might be: + // read index not ready, reason can not read index due to split, region 64037 + // we have paused schedule, but it's temporary, + // if next request takes a long time, there's chance schedule is enabled again + // or on key range border, another engine sharing this region tries to split this + // region may cause this error too. + j.convertStageTo(needRescan) + return false, nil + case errPb.DiskFull != nil: + return false, errors.Errorf("non-retryable error: %s", resp.GetError().GetMessage()) + } + // all others doIngest error, such as stale command, etc. we'll retry it again from writeAndIngestByRange + j.lastRetryableErr = common.ErrKVIngestFailed.GenWithStack(resp.GetError().GetMessage()) + j.convertStageTo(regionScanned) + return false, nil +} diff --git a/br/pkg/lightning/backend/local/region_job_test.go b/br/pkg/lightning/backend/local/region_job_test.go new file mode 100644 index 0000000000000..21723ca116148 --- /dev/null +++ b/br/pkg/lightning/backend/local/region_job_test.go @@ -0,0 +1,183 @@ +// 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 local + +import ( + "context" + "testing" + + "github.com/pingcap/kvproto/pkg/errorpb" + sst "github.com/pingcap/kvproto/pkg/import_sstpb" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/br/pkg/restore/split" + "github.com/stretchr/testify/require" +) + +func TestIsIngestRetryable(t *testing.T) { + ctx := context.Background() + region := &split.RegionInfo{ + Leader: &metapb.Peer{Id: 1}, + Region: &metapb.Region{ + Id: 1, + StartKey: []byte{1}, + EndKey: []byte{3}, + RegionEpoch: &metapb.RegionEpoch{ + ConfVer: 1, + Version: 1, + }, + }, + } + metas := []*sst.SSTMeta{ + { + Range: &sst.Range{ + Start: []byte{1}, + End: []byte{2}, + }, + }, + { + Range: &sst.Range{ + Start: []byte{1, 1}, + End: []byte{2}, + }, + }, + } + job := regionJob{ + stage: wrote, + keyRange: Range{ + start: []byte{1}, + end: []byte{3}, + }, + region: region, + writeResult: &tikvWriteResult{ + sstMeta: metas, + }, + } + splitCli := &mockSplitClient{} + + // NotLeader doesn't mean region peers are changed, so we can retry ingest. + + resp := &sst.IngestResponse{ + Error: &errorpb.Error{ + NotLeader: &errorpb.NotLeader{ + Leader: &metapb.Peer{Id: 2}, + }, + }, + } + + clone := job + canContinueIngest, err := (&clone).fixIngestError(ctx, resp, splitCli) + require.NoError(t, err) + require.True(t, canContinueIngest) + require.Equal(t, wrote, clone.stage) + require.Equal(t, uint64(2), clone.region.Leader.Id) + require.Error(t, clone.lastRetryableErr) + + // EpochNotMatch means region is changed, if the new region covers the old, we can restart the writing process. + // Otherwise, we should restart from region scanning. + + resp.Error = &errorpb.Error{ + EpochNotMatch: &errorpb.EpochNotMatch{ + CurrentRegions: []*metapb.Region{ + { + Id: 1, + StartKey: []byte{1}, + EndKey: []byte{3}, + RegionEpoch: &metapb.RegionEpoch{ + ConfVer: 1, + Version: 2, + }, + Peers: []*metapb.Peer{{Id: 1}}, + }, + }, + }, + } + clone = job + canContinueIngest, err = (&clone).fixIngestError(ctx, resp, splitCli) + require.NoError(t, err) + require.False(t, canContinueIngest) + require.Equal(t, regionScanned, clone.stage) + require.Nil(t, clone.writeResult) + require.Equal(t, uint64(2), clone.region.Region.RegionEpoch.Version) + require.Error(t, clone.lastRetryableErr) + + resp.Error = &errorpb.Error{ + EpochNotMatch: &errorpb.EpochNotMatch{ + CurrentRegions: []*metapb.Region{ + { + Id: 1, + StartKey: []byte{1}, + EndKey: []byte{1, 2}, + RegionEpoch: &metapb.RegionEpoch{ + ConfVer: 1, + Version: 2, + }, + Peers: []*metapb.Peer{{Id: 1}}, + }, + }, + }, + } + clone = job + canContinueIngest, err = (&clone).fixIngestError(ctx, resp, splitCli) + require.NoError(t, err) + require.False(t, canContinueIngest) + require.Equal(t, needRescan, clone.stage) + require.Error(t, clone.lastRetryableErr) + + // TODO: in which case raft layer will drop message? + + resp.Error = &errorpb.Error{Message: "raft: proposal dropped"} + clone = job + canContinueIngest, err = (&clone).fixIngestError(ctx, resp, splitCli) + require.NoError(t, err) + require.False(t, canContinueIngest) + require.Equal(t, needRescan, clone.stage) + require.Error(t, clone.lastRetryableErr) + + // ReadIndexNotReady means the region is changed, we need to restart from region scanning + + resp.Error = &errorpb.Error{ + ReadIndexNotReady: &errorpb.ReadIndexNotReady{ + Reason: "test", + }, + } + clone = job + canContinueIngest, err = (&clone).fixIngestError(ctx, resp, splitCli) + require.NoError(t, err) + require.False(t, canContinueIngest) + require.Equal(t, needRescan, clone.stage) + require.Error(t, clone.lastRetryableErr) + + // TiKV disk full is not retryable + + resp.Error = &errorpb.Error{ + DiskFull: &errorpb.DiskFull{}, + } + clone = job + _, err = (&clone).fixIngestError(ctx, resp, splitCli) + require.ErrorContains(t, err, "non-retryable error") + + // a general error is retryable from writing + + resp.Error = &errorpb.Error{ + StaleCommand: &errorpb.StaleCommand{}, + } + clone = job + canContinueIngest, err = (&clone).fixIngestError(ctx, resp, splitCli) + require.NoError(t, err) + require.False(t, canContinueIngest) + require.Equal(t, regionScanned, clone.stage) + require.Nil(t, clone.writeResult) + require.Error(t, clone.lastRetryableErr) +} diff --git a/br/pkg/lightning/mydump/csv_parser.go b/br/pkg/lightning/mydump/csv_parser.go index f3b9d45a5d989..854f63830b468 100644 --- a/br/pkg/lightning/mydump/csv_parser.go +++ b/br/pkg/lightning/mydump/csv_parser.go @@ -137,7 +137,7 @@ func NewCSVParser( if len(cfg.StartingBy) > 0 { if strings.Contains(cfg.StartingBy, terminator) { - return nil, errors.New("starting-by cannot contain (line) terminator") + return nil, errors.Errorf("STARTING BY '%s' cannot contain LINES TERMINATED BY '%s'", cfg.StartingBy, terminator) } } diff --git a/br/pkg/lightning/mydump/csv_parser_test.go b/br/pkg/lightning/mydump/csv_parser_test.go index 795b11f1c1c20..3e002eeef42ab 100644 --- a/br/pkg/lightning/mydump/csv_parser_test.go +++ b/br/pkg/lightning/mydump/csv_parser_test.go @@ -1346,7 +1346,7 @@ yyy",5,xx"xxxx,8 }, } _, err := mydump.NewCSVParser(context.Background(), &cfg.CSV, nil, 1, ioWorkers, false, nil) - require.ErrorContains(t, err, "starting-by cannot contain (line) terminator") + require.ErrorContains(t, err, "STARTING BY 'x\nxx' cannot contain LINES TERMINATED BY '\n'") } func TestCharsetConversion(t *testing.T) { diff --git a/br/pkg/restore/split/split.go b/br/pkg/restore/split/split.go index 8edf4e16ccf4d..2084df07acf51 100644 --- a/br/pkg/restore/split/split.go +++ b/br/pkg/restore/split/split.go @@ -69,9 +69,9 @@ func CheckRegionConsistency(startKey, endKey []byte, regions []*RegionInfo) erro return nil } -// PaginateScanRegion scan regions with a limit pagination and -// return all regions at once. -// It reduces max gRPC message size. +// PaginateScanRegion scan regions with a limit pagination and return all regions +// at once. The returned regions are continuous and cover the key range. If not, +// or meet errors, it will retry internally. func PaginateScanRegion( ctx context.Context, client SplitClient, startKey, endKey []byte, limit int, ) ([]*RegionInfo, error) { diff --git a/br/pkg/storage/s3.go b/br/pkg/storage/s3.go index 2998098678494..7abbd9d0e814b 100644 --- a/br/pkg/storage/s3.go +++ b/br/pkg/storage/s3.go @@ -157,10 +157,10 @@ func (options *S3BackendOptions) Apply(s3 *backuppb.S3) error { return errors.Trace(err) } if u.Scheme == "" { - return errors.Annotate(berrors.ErrStorageInvalidConfig, "scheme not found in endpoint") + return errors.Errorf("scheme not found in endpoint") } if u.Host == "" { - return errors.Annotate(berrors.ErrStorageInvalidConfig, "host not found in endpoint") + return errors.Errorf("host not found in endpoint") } } // In some cases, we need to set ForcePathStyle to false. diff --git a/br/pkg/streamhelper/advancer.go b/br/pkg/streamhelper/advancer.go index 33c0e0898b66f..b5245d92ad5b2 100644 --- a/br/pkg/streamhelper/advancer.go +++ b/br/pkg/streamhelper/advancer.go @@ -170,33 +170,34 @@ func tsoBefore(n time.Duration) uint64 { return oracle.ComposeTS(now.UnixMilli()-n.Milliseconds(), 0) } +func (c *CheckpointAdvancer) WithCheckpoints(f func(*spans.ValueSortedFull)) { + c.checkpointsMu.Lock() + defer c.checkpointsMu.Unlock() + + f(c.checkpoints) +} + func (c *CheckpointAdvancer) CalculateGlobalCheckpointLight(ctx context.Context, threshold time.Duration) (uint64, error) { var targets []spans.Valued - c.checkpoints.TraverseValuesLessThan(tsoBefore(threshold), func(v spans.Valued) bool { - targets = append(targets, v) - return true + var minValue spans.Valued + c.WithCheckpoints(func(vsf *spans.ValueSortedFull) { + vsf.TraverseValuesLessThan(tsoBefore(threshold), func(v spans.Valued) bool { + targets = append(targets, v) + return true + }) + minValue = vsf.Min() }) + log.Info("[log backup advancer hint] current last region", + zap.Stringer("min", minValue), zap.Int("for-polling", len(targets)), + zap.String("min-ts", oracle.GetTimeFromTS(minValue.Value).Format(time.RFC3339))) if len(targets) == 0 { - c.checkpointsMu.Lock() - defer c.checkpointsMu.Unlock() - return c.checkpoints.MinValue(), nil - } - samples := targets - if len(targets) > 3 { - samples = targets[:3] + return minValue.Value, nil } - for _, sample := range samples { - log.Info("[log backup advancer hint] sample range.", zap.Stringer("sample", sample), zap.Int("total-len", len(targets))) - } - err := c.tryAdvance(ctx, len(targets), func(i int) kv.KeyRange { return targets[i].Key }) if err != nil { return 0, err } - c.checkpointsMu.Lock() - ts := c.checkpoints.MinValue() - c.checkpointsMu.Unlock() - return ts, nil + return minValue.Value, nil } func (c *CheckpointAdvancer) consumeAllTask(ctx context.Context, ch <-chan TaskEvent) error { @@ -289,7 +290,11 @@ func (c *CheckpointAdvancer) onTaskEvent(ctx context.Context, e TaskEvent) error c.taskRange = spans.Collapse(len(e.Ranges), func(i int) kv.KeyRange { return e.Ranges[i] }) c.checkpoints = spans.Sorted(spans.NewFullWith(e.Ranges, 0)) c.lastCheckpoint = e.Info.StartTs - log.Info("added event", zap.Stringer("task", e.Info), zap.Stringer("ranges", logutil.StringifyKeys(c.taskRange))) + p, err := c.env.BlockGCUntil(ctx, c.task.StartTs) + if err != nil { + log.Warn("failed to upload service GC safepoint, skipping.", logutil.ShortError(err)) + } + log.Info("added event", zap.Stringer("task", e.Info), zap.Stringer("ranges", logutil.StringifyKeys(c.taskRange)), zap.Uint64("current-checkpoint", p)) case EventDel: utils.LogBackupTaskCountDec() c.task = nil @@ -303,6 +308,9 @@ func (c *CheckpointAdvancer) onTaskEvent(ctx context.Context, e TaskEvent) error if err := c.env.ClearV3GlobalCheckpointForTask(ctx, e.Name); err != nil { log.Warn("failed to clear global checkpoint", logutil.ShortError(err)) } + if _, err := c.env.BlockGCUntil(ctx, 0); err != nil { + log.Warn("failed to remove service GC safepoint", logutil.ShortError(err)) + } metrics.LastCheckpoint.DeleteLabelValues(e.Name) case EventErr: return e.Err @@ -319,6 +327,7 @@ func (c *CheckpointAdvancer) setCheckpoint(cp uint64) bool { return false } c.lastCheckpoint = cp + metrics.LastCheckpoint.WithLabelValues(c.task.GetName()).Set(float64(c.lastCheckpoint)) return true } @@ -336,7 +345,6 @@ func (c *CheckpointAdvancer) advanceCheckpointBy(ctx context.Context, getCheckpo zap.Uint64("checkpoint", cp), zap.String("task", c.task.Name), zap.Stringer("take", time.Since(start))) - metrics.LastCheckpoint.WithLabelValues(c.task.GetName()).Set(float64(c.lastCheckpoint)) } return nil } @@ -392,6 +400,18 @@ func (c *CheckpointAdvancer) importantTick(ctx context.Context) error { if err := c.env.UploadV3GlobalCheckpointForTask(ctx, c.task.Name, c.lastCheckpoint); err != nil { return errors.Annotate(err, "failed to upload global checkpoint") } + p, err := c.env.BlockGCUntil(ctx, c.lastCheckpoint-1) + if err != nil { + return errors.Annotatef(err, + "failed to update service GC safe point, current checkpoint is %d, target checkpoint is %d", + c.lastCheckpoint-1, p) + } + if p <= c.lastCheckpoint-1 { + log.Info("updated log backup GC safe point.", zap.Uint64("checkpoint", p), zap.Uint64("target", c.lastCheckpoint-1)) + } + if p > c.lastCheckpoint-1 { + log.Warn("update log backup GC safe point failed: stale.", zap.Uint64("checkpoint", p), zap.Uint64("target", c.lastCheckpoint-1)) + } return nil } diff --git a/br/pkg/streamhelper/advancer_env.go b/br/pkg/streamhelper/advancer_env.go index cf27fda7d5c5b..6591ee46e111c 100644 --- a/br/pkg/streamhelper/advancer_env.go +++ b/br/pkg/streamhelper/advancer_env.go @@ -16,6 +16,11 @@ import ( "google.golang.org/grpc/keepalive" ) +const ( + logBackupServiceID = "log-backup-coordinator" + logBackupSafePointTTL = 24 * time.Hour +) + // Env is the interface required by the advancer. type Env interface { // The region scanner provides the region information. @@ -32,6 +37,13 @@ type PDRegionScanner struct { pd.Client } +// Updates the service GC safe point for the cluster. +// Returns the minimal service GC safe point across all services. +// If the arguments is `0`, this would remove the service safe point. +func (c PDRegionScanner) BlockGCUntil(ctx context.Context, at uint64) (uint64, error) { + return c.UpdateServiceGCSafePoint(ctx, logBackupServiceID, int64(logBackupSafePointTTL.Seconds()), at) +} + // RegionScan gets a list of regions, starts from the region that contains key. // Limit limits the maximum number of regions returned. func (c PDRegionScanner) RegionScan(ctx context.Context, key []byte, endKey []byte, limit int) ([]RegionWithLeader, error) { diff --git a/br/pkg/streamhelper/advancer_test.go b/br/pkg/streamhelper/advancer_test.go index 7dd4c71d35b9c..a01a13f96053e 100644 --- a/br/pkg/streamhelper/advancer_test.go +++ b/br/pkg/streamhelper/advancer_test.go @@ -183,6 +183,29 @@ func TestOneStoreFailure(t *testing.T) { require.Equal(t, cp, env.checkpoint) } +func TestGCServiceSafePoint(t *testing.T) { + req := require.New(t) + c := createFakeCluster(t, 4, true) + ctx := context.Background() + c.splitAndScatter("01", "02", "022", "023", "033", "04", "043") + env := &testEnv{fakeCluster: c, testCtx: t} + + adv := streamhelper.NewCheckpointAdvancer(env) + adv.StartTaskListener(ctx) + cp := c.advanceCheckpoints() + c.flushAll() + + req.NoError(adv.OnTick(ctx)) + req.Equal(env.serviceGCSafePoint, cp-1) + + env.unregisterTask() + req.Eventually(func() bool { + env.fakeCluster.mu.Lock() + defer env.fakeCluster.mu.Unlock() + return env.serviceGCSafePoint == 0 + }, 3*time.Second, 100*time.Millisecond) +} + func TestTaskRanges(t *testing.T) { log.SetLevel(zapcore.DebugLevel) c := createFakeCluster(t, 4, true) diff --git a/br/pkg/streamhelper/basic_lib_for_test.go b/br/pkg/streamhelper/basic_lib_for_test.go index 1dff77dd72864..0f8ffabf19e2b 100644 --- a/br/pkg/streamhelper/basic_lib_for_test.go +++ b/br/pkg/streamhelper/basic_lib_for_test.go @@ -91,7 +91,8 @@ type fakeCluster struct { regions []*region testCtx *testing.T - onGetClient func(uint64) error + onGetClient func(uint64) error + serviceGCSafePoint uint64 } func (r *region) splitAt(newID uint64, k string) *region { @@ -242,6 +243,23 @@ func (f *fakeStore) GetLastFlushTSOfRegion(ctx context.Context, in *logbackup.Ge return resp, nil } +// Updates the service GC safe point for the cluster. +// Returns the latest service GC safe point. +// If the arguments is `0`, this would remove the service safe point. +func (f *fakeCluster) BlockGCUntil(ctx context.Context, at uint64) (uint64, error) { + f.mu.Lock() + defer f.mu.Unlock() + if at == 0 { + f.serviceGCSafePoint = at + return at, nil + } + if f.serviceGCSafePoint > at { + return f.serviceGCSafePoint, nil + } + f.serviceGCSafePoint = at + return at, nil +} + // RegionScan gets a list of regions, starts from the region that contains key. // Limit limits the maximum number of regions returned. func (f *fakeCluster) RegionScan(ctx context.Context, key []byte, endKey []byte, limit int) ([]streamhelper.RegionWithLeader, error) { @@ -562,6 +580,7 @@ type testEnv struct { checkpoint uint64 testCtx *testing.T ranges []kv.KeyRange + taskCh chan<- streamhelper.TaskEvent mu sync.Mutex } @@ -580,6 +599,7 @@ func (t *testEnv) Begin(ctx context.Context, ch chan<- streamhelper.TaskEvent) e Ranges: rngs, } ch <- tsk + t.taskCh = ch return nil } @@ -608,3 +628,10 @@ func (t *testEnv) getCheckpoint() uint64 { return t.checkpoint } + +func (t *testEnv) unregisterTask() { + t.taskCh <- streamhelper.TaskEvent{ + Type: streamhelper.EventDel, + Name: "whole", + } +} diff --git a/br/pkg/streamhelper/flush_subscriber.go b/br/pkg/streamhelper/flush_subscriber.go index 70cd4d8e4501d..21a87ac778b30 100644 --- a/br/pkg/streamhelper/flush_subscriber.go +++ b/br/pkg/streamhelper/flush_subscriber.go @@ -277,7 +277,7 @@ func (s *subscription) listenOver(cli eventStream) { Value: m.Checkpoint, } } - metrics.RegionCheckpointSubscriptionEvent.WithLabelValues(strconv.Itoa(int(storeID))).Add(float64(len(msg.Events))) + metrics.RegionCheckpointSubscriptionEvent.WithLabelValues(strconv.Itoa(int(storeID))).Observe(float64(len(msg.Events))) } } diff --git a/br/pkg/streamhelper/regioniter.go b/br/pkg/streamhelper/regioniter.go index 239c710db1ba4..f1ad0e7fb1174 100644 --- a/br/pkg/streamhelper/regioniter.go +++ b/br/pkg/streamhelper/regioniter.go @@ -35,6 +35,13 @@ type TiKVClusterMeta interface { // Stores returns the store metadata from the cluster. Stores(ctx context.Context) ([]Store, error) + + // Updates the service GC safe point for the cluster. + // Returns the latest service GC safe point. + // If the arguments is `0`, this would remove the service safe point. + // NOTE: once we support multi tasks, perhaps we need to allow the caller to provide a namespace. + // For now, all tasks (exactly one task in fact) use the same checkpoint. + BlockGCUntil(ctx context.Context, at uint64) (uint64, error) } type Store struct { diff --git a/br/pkg/streamhelper/regioniter_test.go b/br/pkg/streamhelper/regioniter_test.go index 1c0d6a28ab0fe..c2ad92c9a58da 100644 --- a/br/pkg/streamhelper/regioniter_test.go +++ b/br/pkg/streamhelper/regioniter_test.go @@ -74,6 +74,13 @@ func (c constantRegions) Stores(ctx context.Context) ([]streamhelper.Store, erro return nil, status.Error(codes.Unimplemented, "Unsupported operation") } +// Updates the service GC safe point for the cluster. +// Returns the latest service GC safe point. +// If the arguments is `0`, this would remove the service safe point. +func (c constantRegions) BlockGCUntil(ctx context.Context, at uint64) (uint64, error) { + return 0, status.Error(codes.Unimplemented, "Unsupported operation") +} + func makeSubrangeRegions(keys ...string) constantRegions { if len(keys) == 0 { return nil diff --git a/br/pkg/streamhelper/spans/value_sorted.go b/br/pkg/streamhelper/spans/value_sorted.go index 2fc1ff2cdbbbc..d177fbe765ffa 100644 --- a/br/pkg/streamhelper/spans/value_sorted.go +++ b/br/pkg/streamhelper/spans/value_sorted.go @@ -64,6 +64,10 @@ func (v *ValueSortedFull) TraverseValuesLessThan(n Value, action func(Valued) bo }) } +func (v *ValueSortedFull) Min() Valued { + return Valued(v.valueIdx.Min().(sortedByValueThenStartKey)) +} + func (v *ValueSortedFull) MinValue() Value { return v.valueIdx.Min().(sortedByValueThenStartKey).Value } diff --git a/br/pkg/task/backup.go b/br/pkg/task/backup.go index 5ddaf0e6ccd25..4f8dcd3b578c8 100644 --- a/br/pkg/task/backup.go +++ b/br/pkg/task/backup.go @@ -113,6 +113,10 @@ func DefineBackupFlags(flags *pflag.FlagSet) { "backup sst file compression algorithm, value can be one of 'lz4|zstd|snappy'") flags.Int32(flagCompressionLevel, 0, "compression level used for sst file compression") + flags.Uint32(flagConcurrency, 4, "The size of a BR thread pool that executes tasks, "+ + "One task represents one table range (or one index range) according to the backup schemas. If there is one table with one index."+ + "there will be two tasks to back up this table. This value should increase if you need to back up lots of tables or indices.") + flags.Bool(flagRemoveSchedulers, false, "disable the balance, shuffle and region-merge schedulers in PD to speed up backup") // This flag can impact the online cluster, so hide it in case of abuse. @@ -196,6 +200,10 @@ func (cfg *BackupConfig) ParseFromFlags(flags *pflag.FlagSet) error { log.Info("use checkpoint's default GC TTL", zap.Int64("GC TTL", gcTTL)) } cfg.GCTTL = gcTTL + cfg.Concurrency, err = flags.GetUint32(flagConcurrency) + if err != nil { + return errors.Trace(err) + } compressionCfg, err := parseCompressionFlags(flags) if err != nil { diff --git a/br/pkg/task/common.go b/br/pkg/task/common.go index ad223b7e34c67..3e1fa25d72840 100644 --- a/br/pkg/task/common.go +++ b/br/pkg/task/common.go @@ -253,12 +253,6 @@ func DefineCommonFlags(flags *pflag.FlagSet) { flags.Bool(flagRemoveTiFlash, true, "Remove TiFlash replicas before backup or restore, for unsupported versions of TiFlash") - // Default concurrency is different for backup and restore. - // Leave it 0 and let them adjust the value. - flags.Uint32(flagConcurrency, 0, "The size of thread pool on each node that executes the task") - // It may confuse users , so just hide it. - _ = flags.MarkHidden(flagConcurrency) - flags.Uint64(flagRateLimitUnit, units.MiB, "The unit of rate limit") _ = flags.MarkHidden(flagRateLimitUnit) _ = flags.MarkDeprecated(flagRemoveTiFlash, @@ -471,9 +465,7 @@ func (cfg *Config) ParseFromFlags(flags *pflag.FlagSet) error { if cfg.NoCreds, err = flags.GetBool(flagNoCreds); err != nil { return errors.Trace(err) } - if cfg.Concurrency, err = flags.GetUint32(flagConcurrency); err != nil { - return errors.Trace(err) - } + if cfg.Checksum, err = flags.GetBool(flagChecksum); err != nil { return errors.Trace(err) } diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index 2a5c58a9febf3..e42d4ab877ab1 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -117,7 +117,8 @@ func (cfg *RestoreCommonConfig) adjust() { func DefineRestoreCommonFlags(flags *pflag.FlagSet) { // TODO remove experimental tag if it's stable flags.Bool(flagOnline, false, "(experimental) Whether online when restore") - + flags.Uint32(flagConcurrency, 128, "The size of thread pool on BR that executes tasks, "+ + "where each task restores one SST file to TiKV") flags.Uint64(FlagMergeRegionSizeBytes, conn.DefaultMergeRegionSizeBytes, "the threshold of merging small regions (Default 96MB, region split size)") flags.Uint64(FlagMergeRegionKeyCount, conn.DefaultMergeRegionKeyCount, @@ -153,6 +154,7 @@ func (cfg *RestoreCommonConfig) ParseFromFlags(flags *pflag.FlagSet) error { if err != nil { return errors.Trace(err) } + if flags.Lookup(flagWithSysTable) != nil { cfg.WithSysTable, err = flags.GetBool(flagWithSysTable) if err != nil { @@ -280,7 +282,10 @@ func (cfg *RestoreConfig) ParseFromFlags(flags *pflag.FlagSet) error { if err != nil { return errors.Trace(err) } - + cfg.Concurrency, err = flags.GetUint32(flagConcurrency) + if err != nil { + return errors.Trace(err) + } if cfg.Config.Concurrency == 0 { cfg.Config.Concurrency = defaultRestoreConcurrency } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index b52ead50806c7..9f2179dbc7932 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2150,7 +2150,7 @@ func checkPartitionDefinitionConstraints(ctx sessionctx.Context, tbInfo *model.T switch tbInfo.Partition.Type { case model.PartitionTypeRange: err = checkPartitionByRange(ctx, tbInfo) - case model.PartitionTypeHash: + case model.PartitionTypeHash, model.PartitionTypeKey: err = checkPartitionByHash(ctx, tbInfo) case model.PartitionTypeList: err = checkPartitionByList(ctx, tbInfo) @@ -2873,7 +2873,21 @@ func checkPartitionByList(ctx sessionctx.Context, tbInfo *model.TableInfo) error return checkListPartitionValue(ctx, tbInfo) } -func isColTypeAllowedAsPartitioningCol(fieldType types.FieldType) bool { +func isValidKeyPartitionColType(fieldType types.FieldType) bool { + switch fieldType.GetType() { + case mysql.TypeBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeJSON, mysql.TypeGeometry: + return false + default: + return true + } +} + +func isColTypeAllowedAsPartitioningCol(partType model.PartitionType, fieldType types.FieldType) bool { + // For key partition, the permitted partition field types can be all field types except + // BLOB, JSON, Geometry + if partType == model.PartitionTypeKey { + return isValidKeyPartitionColType(fieldType) + } // The permitted data types are shown in the following list: // All integer types // DATE and DATETIME @@ -2896,7 +2910,7 @@ func checkColumnsPartitionType(tbInfo *model.TableInfo) error { if colInfo == nil { return errors.Trace(dbterror.ErrFieldNotFoundPart) } - if !isColTypeAllowedAsPartitioningCol(colInfo.FieldType) { + if !isColTypeAllowedAsPartitioningCol(tbInfo.Partition.Type, colInfo.FieldType) { return dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(col.O) } } @@ -4112,13 +4126,12 @@ func (d *ddl) CoalescePartitions(ctx sessionctx.Context, ident ast.Ident, spec * // Key type partition cannot be constructed currently, ignoring it for now. case model.PartitionTypeKey: + return errors.Trace(dbterror.ErrUnsupportedCoalescePartition) // Coalesce partition can only be used on hash/key partitions. default: return errors.Trace(dbterror.ErrCoalesceOnlyOnHashPartition) } - - return errors.Trace(err) } func (d *ddl) TruncateTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { @@ -4965,7 +4978,7 @@ func GetModifiableColumnJob( if col.Name.L != newCol.Name.L { return nil, dbterror.ErrDependentByPartitionFunctional.GenWithStackByArgs(col.Name.L) } - if !isColTypeAllowedAsPartitioningCol(newCol.FieldType) { + if !isColTypeAllowedAsPartitioningCol(t.Meta().Partition.Type, newCol.FieldType) { return nil, dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(newCol.Name.O) } pi := pt.Meta().GetPartitionInfo() @@ -5860,7 +5873,6 @@ func checkAlterTableCharset(tblInfo *model.TableInfo, dbInfo *model.DBInfo, toCh } } - // The table charset may be "", if the table is create in old TiDB version, such as v2.0.8. // This DDL will update the table charset to default charset. origCharset, origCollate, err = ResolveCharsetCollation( ast.CharsetOpt{Chs: origCharset, Col: origCollate}, diff --git a/ddl/index.go b/ddl/index.go index f9bfeba446855..ef8c157a06f76 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -1634,6 +1634,11 @@ func genKeyExistsErr(key, value []byte, idxInfo *model.IndexInfo, tblInfo *model } func (w *addIndexWorker) batchCheckUniqueKey(txn kv.Transaction, idxRecords []*indexRecord) error { + if w.writerCtx != nil { + // For the ingest mode, we use lightning local backend's local check and remote check + // to implement the duplicate key detection. + return nil + } idxInfo := w.index.Meta() if !idxInfo.Unique { // non-unique key need not to check, just overwrite it, diff --git a/ddl/job_table.go b/ddl/job_table.go index 6d5adecc09614..0ce54414d8f5b 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -19,7 +19,6 @@ import ( "context" "encoding/hex" "fmt" - "math" "strconv" "strings" "time" @@ -537,20 +536,6 @@ func getDDLReorgHandle(sess *session, job *model.Job) (element *meta.Element, st startKey = rows[0].GetBytes(2) endKey = rows[0].GetBytes(3) physicalTableID = rows[0].GetInt64(4) - // physicalTableID may be 0, because older version TiDB (without table partition) doesn't store them. - // update them to table's in this case. - if physicalTableID == 0 { - if job.ReorgMeta != nil { - endKey = kv.IntHandle(job.ReorgMeta.EndHandle).Encoded() - } else { - endKey = kv.IntHandle(math.MaxInt64).Encoded() - } - physicalTableID = job.TableID - logutil.BgLogger().Warn("new TiDB binary running on old TiDB DDL reorg data", - zap.Int64("partition ID", physicalTableID), - zap.Stringer("startHandle", startKey), - zap.Stringer("endHandle", endKey)) - } return } diff --git a/ddl/partition.go b/ddl/partition.go index c3f0dba0e1564..955757808f779 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -452,6 +452,14 @@ func buildTablePartitionInfo(ctx sessionctx.Context, s *ast.PartitionOptions, tb if s.Sub == nil { enable = true } + case model.PartitionTypeKey: + // Note that linear key is simply ignored, and creates non-linear hash. + if s.Linear { + ctx.GetSessionVars().StmtCtx.AppendWarning(dbterror.ErrUnsupportedCreatePartition.GenWithStack("LINEAR KEY is not supported, using non-linear KEY instead")) + } + if s.Sub == nil && len(s.ColumnNames) != 0 { + enable = true + } case model.PartitionTypeList: // Partition by list is enabled only when tidb_enable_list_partition is 'ON'. enable = ctx.GetSessionVars().EnableListTablePartition @@ -1075,7 +1083,7 @@ func buildPartitionDefinitionsInfo(ctx sessionctx.Context, defs []*ast.Partition switch tbInfo.Partition.Type { case model.PartitionTypeRange: partitions, err = buildRangePartitionDefinitions(ctx, defs, tbInfo) - case model.PartitionTypeHash: + case model.PartitionTypeHash, model.PartitionTypeKey: partitions, err = buildHashPartitionDefinitions(ctx, defs, tbInfo) case model.PartitionTypeList: partitions, err = buildListPartitionDefinitions(ctx, defs, tbInfo) @@ -3553,6 +3561,15 @@ func hexIfNonPrint(s string) string { return "0x" + hex.EncodeToString([]byte(driver.UnwrapFromSingleQuotes(s))) } +func writeColumnListToBuffer(partitionInfo *model.PartitionInfo, sqlMode mysql.SQLMode, buf *bytes.Buffer) { + for i, col := range partitionInfo.Columns { + buf.WriteString(stringutil.Escape(col.O, sqlMode)) + if i < len(partitionInfo.Columns)-1 { + buf.WriteString(",") + } + } +} + // AppendPartitionInfo is used in SHOW CREATE TABLE as well as generation the SQL syntax // for the PartitionInfo during validation of various DDL commands func AppendPartitionInfo(partitionInfo *model.PartitionInfo, buf *bytes.Buffer, sqlMode mysql.SQLMode) { @@ -3563,8 +3580,9 @@ func AppendPartitionInfo(partitionInfo *model.PartitionInfo, buf *bytes.Buffer, // include the /*!50100 or /*!50500 comments for TiDB. // This also solves the issue with comments within comments that would happen for // PLACEMENT POLICY options. - if partitionInfo.Type == model.PartitionTypeHash { - defaultPartitionDefinitions := true + defaultPartitionDefinitions := true + if partitionInfo.Type == model.PartitionTypeHash || + partitionInfo.Type == model.PartitionTypeKey { for i, def := range partitionInfo.Definitions { if def.Name.O != fmt.Sprintf("p%d", i) { defaultPartitionDefinitions = false @@ -3577,21 +3595,28 @@ func AppendPartitionInfo(partitionInfo *model.PartitionInfo, buf *bytes.Buffer, } if defaultPartitionDefinitions { - fmt.Fprintf(buf, "\nPARTITION BY HASH (%s) PARTITIONS %d", partitionInfo.Expr, partitionInfo.Num) + if partitionInfo.Type == model.PartitionTypeHash { + fmt.Fprintf(buf, "\nPARTITION BY HASH (%s) PARTITIONS %d", partitionInfo.Expr, partitionInfo.Num) + } else { + buf.WriteString("\nPARTITION BY KEY (") + writeColumnListToBuffer(partitionInfo, sqlMode, buf) + buf.WriteString(")") + fmt.Fprintf(buf, " PARTITIONS %d", partitionInfo.Num) + } return } } - // this if statement takes care of lists/range columns case + // this if statement takes care of lists/range/key columns case if len(partitionInfo.Columns) > 0 { // partitionInfo.Type == model.PartitionTypeRange || partitionInfo.Type == model.PartitionTypeList + // || partitionInfo.Type == model.PartitionTypeKey // Notice that MySQL uses two spaces between LIST and COLUMNS... - fmt.Fprintf(buf, "\nPARTITION BY %s COLUMNS(", partitionInfo.Type.String()) - for i, col := range partitionInfo.Columns { - buf.WriteString(stringutil.Escape(col.O, sqlMode)) - if i < len(partitionInfo.Columns)-1 { - buf.WriteString(",") - } + if partitionInfo.Type == model.PartitionTypeKey { + fmt.Fprintf(buf, "\nPARTITION BY %s (", partitionInfo.Type.String()) + } else { + fmt.Fprintf(buf, "\nPARTITION BY %s COLUMNS(", partitionInfo.Type.String()) } + writeColumnListToBuffer(partitionInfo, sqlMode, buf) buf.WriteString(")\n(") } else { fmt.Fprintf(buf, "\nPARTITION BY %s (%s)\n(", partitionInfo.Type.String(), partitionInfo.Expr) @@ -3610,7 +3635,7 @@ func AppendPartitionDefs(partitionInfo *model.PartitionInfo, buf *bytes.Buffer, fmt.Fprintf(buf, ",\n ") } fmt.Fprintf(buf, "PARTITION %s", stringutil.Escape(def.Name.O, sqlMode)) - // PartitionTypeHash does not have any VALUES definition + // PartitionTypeHash and PartitionTypeKey do not have any VALUES definition if partitionInfo.Type == model.PartitionTypeRange { lessThans := make([]string, len(def.LessThan)) for idx, v := range def.LessThan { diff --git a/domain/extract.go b/domain/extract.go index 64c7dc2713c2a..896a65af8ed53 100644 --- a/domain/extract.go +++ b/domain/extract.go @@ -23,6 +23,7 @@ import ( "os" "path/filepath" "strings" + "sync" "time" "github.com/BurntSushi/toml" @@ -92,6 +93,7 @@ type extractWorker struct { ctx context.Context sctx sessionctx.Context isBackgroundWorker bool + sync.Mutex } // ExtractTask indicates task @@ -146,6 +148,8 @@ func (w *extractWorker) extractPlanTask(ctx context.Context, task *ExtractTask) } func (w *extractWorker) collectRecords(ctx context.Context, task *ExtractTask) (map[stmtSummaryHistoryKey]stmtSummaryHistoryRecord, error) { + w.Lock() + defer w.Unlock() exec := w.sctx.(sqlexec.RestrictedSQLExecutor) ctx1 := kv.WithInternalSourceType(ctx, kv.InternalTxnStats) rows, _, err := exec.ExecRestrictedSQL(ctx1, nil, fmt.Sprintf("SELECT STMT_TYPE, TABLE_NAMES, DIGEST, PLAN_DIGEST,QUERY_SAMPLE_TEXT, BINARY_PLAN FROM INFORMATION_SCHEMA.STATEMENTS_SUMMARY_HISTORY WHERE SUMMARY_END_TIME > '%s' OR SUMMARY_BEGIN_TIME < '%s'", diff --git a/errno/errcode.go b/errno/errcode.go index 0501bc0020e07..a165e2f6ded2e 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -1045,6 +1045,15 @@ const ( ErrTempTableNotAllowedWithTTL = 8151 ErrUnsupportedTTLReferencedByFK = 8152 ErrUnsupportedPrimaryKeyTypeWithTTL = 8153 + ErrLoadDataFromServerDisk = 8154 + ErrLoadParquetFromLocal = 8155 + ErrLoadDataEmptyPath = 8156 + ErrLoadDataUnsupportedFormat = 8157 + ErrLoadDataInvalidURI = 8158 + ErrLoadDataCantAccess = 8159 + ErrLoadDataCantRead = 8160 + ErrLoadDataPhysicalImportTableNotEmpty = 8161 + ErrLoadDataWrongFormatConfig = 8162 // Error codes used by TiDB ddl package ErrUnsupportedDDLOperation = 8200 diff --git a/errno/errname.go b/errno/errname.go index 1d5a5242ca227..b7272a4710e27 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -1040,6 +1040,14 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrTempTableNotAllowedWithTTL: mysql.Message("Set TTL for temporary table is not allowed", nil), ErrUnsupportedTTLReferencedByFK: mysql.Message("Set TTL for a table referenced by foreign key is not allowed", nil), ErrUnsupportedPrimaryKeyTypeWithTTL: mysql.Message("Unsupported clustered primary key type FLOAT/DOUBLE for TTL", nil), + ErrLoadDataFromServerDisk: mysql.Message("Don't support load data from tidb-server's disk. Or if you want to load local data via client, the path of INFILE '%s' needs to specify the clause of LOCAL first", nil), + ErrLoadParquetFromLocal: mysql.Message("Do not support loading parquet files from local. Please try to load the parquet files from the cloud storage", nil), + ErrLoadDataEmptyPath: mysql.Message("The value of INFILE must not be empty when LOAD DATA from LOCAL", nil), + ErrLoadDataUnsupportedFormat: mysql.Message("The FORMAT '%s' is not supported", nil), + ErrLoadDataInvalidURI: mysql.Message("The URI of INFILE is invalid. Reason: %s. Please provide a valid URI, such as 's3://import/test.csv?access_key_id={your_access_key_id ID}&secret_access_key={your_secret_access_key}&session_token={your_session_token}'", nil), + ErrLoadDataCantAccess: mysql.Message("Access to the source file has been denied. Please check the URI, access key and secret access key are correct", nil), + ErrLoadDataCantRead: mysql.Message("Failed to read source files. Reason: %s. %s", nil), + ErrLoadDataWrongFormatConfig: mysql.Message("", nil), ErrWarnOptimizerHintInvalidInteger: mysql.Message("integer value is out of range in '%s'", nil), ErrWarnOptimizerHintUnsupportedHint: mysql.Message("Optimizer hint %s is not supported by TiDB and is ignored", nil), diff --git a/errors.toml b/errors.toml index 4bf81f9ef7866..bbc2ff8d78192 100644 --- a/errors.toml +++ b/errors.toml @@ -1646,6 +1646,46 @@ error = ''' transaction aborted because lazy uniqueness check is enabled and an error occurred: %s ''' +["executor:8154"] +error = ''' +Don't support load data from tidb-server's disk. Or if you want to load local data via client, the path of INFILE '%s' needs to specify the clause of LOCAL first +''' + +["executor:8155"] +error = ''' +Do not support loading parquet files from local. Please try to load the parquet files from the cloud storage +''' + +["executor:8156"] +error = ''' +The value of INFILE must not be empty when LOAD DATA from LOCAL +''' + +["executor:8157"] +error = ''' +The FORMAT '%s' is not supported +''' + +["executor:8158"] +error = ''' +The URI of INFILE is invalid. Reason: %s. Please provide a valid URI, such as 's3://import/test.csv?access_key_id={your_access_key_id ID}&secret_access_key={your_secret_access_key}&session_token={your_session_token}' +''' + +["executor:8159"] +error = ''' +Access to the source file has been denied. Please check the URI, access key and secret access key are correct +''' + +["executor:8160"] +error = ''' +Failed to read source files. Reason: %s. %s +''' + +["executor:8162"] +error = ''' + +''' + ["executor:8212"] error = ''' Failed to split region ranges: %s diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index 52d3fc5986b4b..4393a91b32e64 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -347,6 +347,7 @@ go_test( flaky = True, shard_count = 50, deps = [ + "//br/pkg/errors", "//config", "//ddl", "//ddl/placement", diff --git a/executor/adapter.go b/executor/adapter.go index 7a4649ed3f8b5..133e54518757e 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -883,17 +883,24 @@ func (c *chunkRowRecordSet) Close() error { return nil } -func (a *ExecStmt) handlePessimisticSelectForUpdate(ctx context.Context, e Executor) (sqlexec.RecordSet, error) { +func (a *ExecStmt) handlePessimisticSelectForUpdate(ctx context.Context, e Executor) (_ sqlexec.RecordSet, retErr error) { if snapshotTS := a.Ctx.GetSessionVars().SnapshotTS; snapshotTS != 0 { terror.Log(e.Close()) return nil, errors.New("can not execute write statement when 'tidb_snapshot' is set") } txnManager := sessiontxn.GetTxnManager(a.Ctx) - err := txnManager.OnHandlePessimisticStmtStart(ctx) + err := txnManager.OnPessimisticStmtStart(ctx) if err != nil { return nil, err } + defer func() { + isSuccessful := retErr == nil + err1 := txnManager.OnPessimisticStmtEnd(ctx, isSuccessful) + if retErr == nil && err1 != nil { + retErr = err1 + } + }() isFirstAttempt := true @@ -1011,10 +1018,17 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) (err er }() txnManager := sessiontxn.GetTxnManager(a.Ctx) - err = txnManager.OnHandlePessimisticStmtStart(ctx) + err = txnManager.OnPessimisticStmtStart(ctx) if err != nil { return err } + defer func() { + isSuccessful := err == nil + err1 := txnManager.OnPessimisticStmtEnd(ctx, isSuccessful) + if err == nil && err1 != nil { + err = err1 + } + }() isFirstAttempt := true @@ -1104,7 +1118,7 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error }() txnManager := sessiontxn.GetTxnManager(a.Ctx) - action, err := txnManager.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) + action, err := txnManager.OnStmtErrorForNextAction(ctx, sessiontxn.StmtErrAfterPessimisticLock, lockErr) if err != nil { return nil, err } diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index ee9808700aaec..f4a85c5214d21 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -226,7 +226,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { continue } - physID, err := getPhysID(e.tblInfo, e.partExpr, idxVals[e.partPos].GetInt64()) + physID, err := getPhysID(e.tblInfo, e.partExpr, idxVals[e.partPos]) if err != nil { continue } @@ -357,7 +357,8 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { tID = e.physIDs[i] } else { if handle.IsInt() { - tID, err = getPhysID(e.tblInfo, e.partExpr, handle.IntValue()) + d := types.NewIntDatum(handle.IntValue()) + tID, err = getPhysID(e.tblInfo, e.partExpr, d) if err != nil { continue } @@ -366,7 +367,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { if err1 != nil { return err1 } - tID, err = getPhysID(e.tblInfo, e.partExpr, d.GetInt64()) + tID, err = getPhysID(e.tblInfo, e.partExpr, d) if err != nil { continue } @@ -527,7 +528,7 @@ func (getter *PessimisticLockCacheGetter) Get(_ context.Context, key kv.Key) ([] return nil, kv.ErrNotExist } -func getPhysID(tblInfo *model.TableInfo, partitionExpr *tables.PartitionExpr, intVal int64) (int64, error) { +func getPhysID(tblInfo *model.TableInfo, partitionExpr *tables.PartitionExpr, d types.Datum) (int64, error) { pi := tblInfo.GetPartitionInfo() if pi == nil { return tblInfo.ID, nil @@ -539,8 +540,18 @@ func getPhysID(tblInfo *model.TableInfo, partitionExpr *tables.PartitionExpr, in switch pi.Type { case model.PartitionTypeHash: + intVal := d.GetInt64() partIdx := mathutil.Abs(intVal % int64(pi.Num)) return pi.Definitions[partIdx].ID, nil + case model.PartitionTypeKey: + if len(pi.Columns) > 1 { + return 0, errors.Errorf("unsupported partition type in BatchGet") + } + partIdx, err := partitionExpr.LocateKeyPartitionWithSPC(pi, []types.Datum{d}) + if err != nil { + return 0, errors.Errorf("unsupported partition type in BatchGet") + } + return pi.Definitions[partIdx].ID, nil case model.PartitionTypeRange: // we've check the type assertions in func TryFastPlan col, ok := partitionExpr.Expr.(*expression.Column) @@ -550,6 +561,7 @@ func getPhysID(tblInfo *model.TableInfo, partitionExpr *tables.PartitionExpr, in unsigned := mysql.HasUnsignedFlag(col.GetType().GetFlag()) ranges := partitionExpr.ForRangePruning length := len(ranges.LessThan) + intVal := d.GetInt64() partIdx := sort.Search(length, func(i int) bool { return ranges.Compare(i, intVal, unsigned) > 0 }) @@ -558,6 +570,7 @@ func getPhysID(tblInfo *model.TableInfo, partitionExpr *tables.PartitionExpr, in } case model.PartitionTypeList: isNull := false // we've guaranteed this in the build process of either TryFastPlan or buildBatchPointGet + intVal := d.GetInt64() partIdx := partitionExpr.ForListPruning.LocatePartition(intVal, isNull) if partIdx >= 0 { return pi.Definitions[partIdx].ID, nil diff --git a/executor/builder.go b/executor/builder.go index 18ab4913aadd5..b38e5d85148bb 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -938,46 +938,23 @@ func (b *executorBuilder) buildLoadData(v *plannercore.LoadData) Executor { b.err = errors.Errorf("Can not get table %d", v.Table.TableInfo.ID) return nil } - insertVal := &InsertValues{ - baseExecutor: newBaseExecutor(b.ctx, nil, v.ID()), - Table: tbl, - Columns: v.Columns, - GenExprs: v.GenCols.Exprs, - isLoadData: true, - txnInUse: sync.Mutex{}, - } - loadDataInfo := &LoadDataInfo{ - row: make([]types.Datum, 0, len(insertVal.insertColumns)), - InsertValues: insertVal, - Path: v.Path, - Format: v.Format, - Table: tbl, - FieldsInfo: v.FieldsInfo, - LinesInfo: v.LinesInfo, - NullInfo: v.NullInfo, - IgnoreLines: v.IgnoreLines, - ColumnAssignments: v.ColumnAssignments, - ColumnsAndUserVars: v.ColumnsAndUserVars, - OnDuplicate: v.OnDuplicate, - Ctx: b.ctx, - } - columnNames := loadDataInfo.initFieldMappings() - err := loadDataInfo.initLoadColumns(columnNames) + if !tbl.Meta().IsBaseTable() { + b.err = plannercore.ErrNonUpdatableTable.GenWithStackByArgs(tbl.Meta().Name.O, "LOAD") + return nil + } + + worker, err := NewLoadDataWorker(b.ctx, v, tbl) if err != nil { b.err = err return nil } - loadDataExec := &LoadDataExec{ - baseExecutor: newBaseExecutor(b.ctx, nil, v.ID()), - FileLocRef: v.FileLocRef, - OnDuplicate: v.OnDuplicate, - loadDataInfo: loadDataInfo, - } - var defaultLoadDataBatchCnt uint64 = 20000 // TODO this will be changed to variable in another pr - loadDataExec.loadDataInfo.initQueues() - loadDataExec.loadDataInfo.SetMaxRowsInBatch(defaultLoadDataBatchCnt) - return loadDataExec + return &LoadDataExec{ + baseExecutor: newBaseExecutor(b.ctx, nil, v.ID()), + FileLocRef: v.FileLocRef, + OnDuplicate: v.OnDuplicate, + loadDataWorker: worker, + } } func (b *executorBuilder) buildLoadStats(v *plannercore.LoadStats) Executor { diff --git a/executor/errors.go b/executor/errors.go index 565a712d1c7d9..45c6916bbe146 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -76,4 +76,13 @@ var ( errUnsupportedFlashbackTmpTable = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("Recover/flashback table is not supported on temporary tables", nil)) errTruncateWrongInsertValue = dbterror.ClassTable.NewStdErr(mysql.ErrTruncatedWrongValue, parser_mysql.Message("Incorrect %-.32s value: '%-.128s' for column '%.192s' at row %d", nil)) ErrExistsInHistoryPassword = dbterror.ClassExecutor.NewStd(mysql.ErrExistsInHistoryPassword) + + ErrLoadDataFromServerDisk = dbterror.ClassExecutor.NewStd(mysql.ErrLoadDataFromServerDisk) + ErrLoadParquetFromLocal = dbterror.ClassExecutor.NewStd(mysql.ErrLoadParquetFromLocal) + ErrLoadDataEmptyPath = dbterror.ClassExecutor.NewStd(mysql.ErrLoadDataEmptyPath) + ErrLoadDataUnsupportedFormat = dbterror.ClassExecutor.NewStd(mysql.ErrLoadDataUnsupportedFormat) + ErrLoadDataInvalidURI = dbterror.ClassExecutor.NewStd(mysql.ErrLoadDataInvalidURI) + ErrLoadDataCantAccess = dbterror.ClassExecutor.NewStd(mysql.ErrLoadDataCantAccess) + ErrLoadDataCantRead = dbterror.ClassExecutor.NewStd(mysql.ErrLoadDataCantRead) + ErrLoadDataWrongFormatConfig = dbterror.ClassExecutor.NewStd(mysql.ErrLoadDataWrongFormatConfig) ) diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index bbc17fd0aa3f7..8415169d10c88 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -464,9 +464,8 @@ func TestPointGetUserVarPlanCache(t *testing.T) { tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // can use idx_a `Projection_9 1.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b`, `└─MergeJoin_10 1.00 root inner join, left key:test.t2.a, right key:test.t1.a`, - ` ├─Selection_42(Build) 10.00 root eq(test.t1.a, 1)`, - ` │ └─TableReader_41 10.00 root data:TableRangeScan_40`, - ` │ └─TableRangeScan_40 10.00 cop[tikv] table:t1 range:[1,1], keep order:true, stats:pseudo`, + ` ├─TableReader_41(Build) 10.00 root data:TableRangeScan_40`, + ` │ └─TableRangeScan_40 10.00 cop[tikv] table:t1 range:[1,1], keep order:true, stats:pseudo`, ` └─Selection_39(Probe) 0.80 root not(isnull(test.t2.a))`, ` └─Point_Get_38 1.00 root table:t2, index:idx_a(a) `)) @@ -480,9 +479,8 @@ func TestPointGetUserVarPlanCache(t *testing.T) { tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // can use idx_a `Projection_9 1.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b`, `└─MergeJoin_10 1.00 root inner join, left key:test.t2.a, right key:test.t1.a`, - ` ├─Selection_42(Build) 10.00 root eq(test.t1.a, 2)`, - ` │ └─TableReader_41 10.00 root data:TableRangeScan_40`, - ` │ └─TableRangeScan_40 10.00 cop[tikv] table:t1 range:[2,2], keep order:true, stats:pseudo`, + ` ├─TableReader_41(Build) 10.00 root data:TableRangeScan_40`, + ` │ └─TableRangeScan_40 10.00 cop[tikv] table:t1 range:[2,2], keep order:true, stats:pseudo`, ` └─Selection_39(Probe) 0.80 root not(isnull(test.t2.a))`, ` └─Point_Get_38 1.00 root table:t2, index:idx_a(a) `)) tk.MustQuery("execute stmt using @a").Check(testkit.Rows( @@ -539,9 +537,8 @@ func TestIssue28259(t *testing.T) { ps := []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res := tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Len(t, res.Rows(), 3) - require.Regexp(t, ".*Selection.*", res.Rows()[0][0]) - require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[2][0]) + require.Len(t, res.Rows(), 2) + require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[1][0]) tk.MustExec("set @a=-1696020282760139948, @b=-2619168038882941276, @c=-4004648990067362699;") tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows()) @@ -575,11 +572,9 @@ func TestIssue28259(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Len(t, res.Rows(), 5) - require.Regexp(t, ".*Selection.*", res.Rows()[1][0]) - require.Equal(t, "lt(test.t.b, 1), or(and(ge(test.t.a, 0), le(test.t.a, 2)), lt(test.t.a, 2))", res.Rows()[1][4]) - require.Regexp(t, ".*IndexReader.*", res.Rows()[2][0]) - require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[4][0]) + require.Len(t, res.Rows(), 4) + require.Regexp(t, ".*IndexReader.*", res.Rows()[1][0]) + require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[3][0]) tk.MustExec("set @a=2, @b=1, @c=1;") tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows()) @@ -619,12 +614,11 @@ func TestIssue28259(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Len(t, res.Rows(), 6) - require.Regexp(t, ".*Selection.*", res.Rows()[1][0]) - require.Regexp(t, ".*IndexLookUp.*", res.Rows()[2][0]) - require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[3][0]) - require.Regexp(t, ".*Selection.*", res.Rows()[4][0]) - require.Regexp(t, ".*TableRowIDScan.*", res.Rows()[5][0]) + require.Len(t, res.Rows(), 5) + require.Regexp(t, ".*IndexLookUp.*", res.Rows()[1][0]) + require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[2][0]) + require.Regexp(t, ".*Selection.*", res.Rows()[3][0]) + require.Regexp(t, ".*TableRowIDScan.*", res.Rows()[4][0]) tk.MustExec("set @a=2, @b=1, @c=1;") tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows()) @@ -662,12 +656,10 @@ func TestIssue28259(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Len(t, res.Rows(), 5) - require.Regexp(t, ".*Selection.*", res.Rows()[1][0]) - require.Equal(t, "lt(test.t.b, 1), or(and(ge(test.t.a, 0), le(test.t.a, 2)), lt(test.t.a, 2))", res.Rows()[1][4]) - require.Regexp(t, ".*TableReader.*", res.Rows()[2][0]) - require.Regexp(t, ".*Selection.*", res.Rows()[3][0]) - require.Regexp(t, ".*TableRangeScan.*", res.Rows()[4][0]) + require.Len(t, res.Rows(), 4) + require.Regexp(t, ".*TableReader.*", res.Rows()[1][0]) + require.Regexp(t, ".*Selection.*", res.Rows()[2][0]) + require.Regexp(t, ".*TableRangeScan.*", res.Rows()[3][0]) tk.MustExec("set @a=2, @b=1, @c=1;") tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows()) @@ -677,12 +669,10 @@ func TestIssue28259(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Len(t, res.Rows(), 5) - require.Regexp(t, ".*Selection.*", res.Rows()[1][0]) - require.Equal(t, "lt(test.t.b, 1), or(and(ge(test.t.a, 2), le(test.t.a, 1)), lt(test.t.a, 1))", res.Rows()[1][4]) - require.Regexp(t, ".*TableReader.*", res.Rows()[2][0]) - require.Regexp(t, ".*Selection.*", res.Rows()[3][0]) - require.Regexp(t, ".*TableRangeScan.*", res.Rows()[4][0]) + require.Len(t, res.Rows(), 4) + require.Regexp(t, ".*TableReader.*", res.Rows()[1][0]) + require.Regexp(t, ".*Selection.*", res.Rows()[2][0]) + require.Regexp(t, ".*TableRangeScan.*", res.Rows()[3][0]) res = tk.MustQuery("explain format = 'brief' select a from t " + "where (a between 0 and 2 or a < 2) and b < 1;") @@ -708,11 +698,10 @@ func TestIssue28259(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Len(t, res.Rows(), 4) - require.Regexp(t, ".*Selection.*", res.Rows()[0][0]) - require.Regexp(t, ".*TableReader.*", res.Rows()[1][0]) - require.Regexp(t, ".*Selection.*", res.Rows()[2][0]) - require.Regexp(t, ".*TableRangeScan.*", res.Rows()[3][0]) + require.Len(t, res.Rows(), 3) + require.Regexp(t, ".*TableReader.*", res.Rows()[0][0]) + require.Regexp(t, ".*Selection.*", res.Rows()[1][0]) + require.Regexp(t, ".*TableRangeScan.*", res.Rows()[2][0]) } func TestIssue28696(t *testing.T) { @@ -734,12 +723,11 @@ func TestIssue28696(t *testing.T) { ps := []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res := tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Len(t, res.Rows(), 6) - require.Regexp(t, ".*Selection.*", res.Rows()[1][0]) - require.Regexp(t, ".*IndexLookUp.*", res.Rows()[2][0]) - require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[3][0]) - require.Regexp(t, ".*Selection.*", res.Rows()[4][0]) - require.Regexp(t, ".*TableRowIDScan.*", res.Rows()[5][0]) + require.Len(t, res.Rows(), 5) + require.Regexp(t, ".*IndexLookUp.*", res.Rows()[1][0]) + require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[2][0]) + require.Regexp(t, ".*Selection.*", res.Rows()[3][0]) + require.Regexp(t, ".*TableRowIDScan.*", res.Rows()[4][0]) res = tk.MustQuery("explain format = 'brief' select a from t1 where b = 'bbcsa';") require.Len(t, res.Rows(), 5) @@ -780,13 +768,12 @@ func TestIndexMerge4PlanCache(t *testing.T) { ps := []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res := tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Len(t, res.Rows(), 7) - require.Regexp(t, ".*Selection.*", res.Rows()[1][0]) - require.Regexp(t, ".*IndexMerge.*", res.Rows()[2][0]) + require.Len(t, res.Rows(), 6) + require.Regexp(t, ".*IndexMerge.*", res.Rows()[1][0]) + require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[3][0]) + require.Equal(t, "range:(NULL,\"mm\"), (\"mm\",+inf], keep order:false, stats:pseudo", res.Rows()[3][4]) require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[4][0]) - require.Equal(t, "range:(NULL,\"mm\"), (\"mm\",+inf], keep order:false, stats:pseudo", res.Rows()[4][4]) - require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[5][0]) - require.Equal(t, "range:[0198-09-29 20:19:49,0198-09-29 20:19:49], keep order:false, stats:pseudo", res.Rows()[5][4]) + require.Equal(t, "range:[0198-09-29 20:19:49,0198-09-29 20:19:49], keep order:false, stats:pseudo", res.Rows()[4][4]) // test for cluster index in indexMerge tk.MustExec("drop table if exists t;") @@ -800,13 +787,12 @@ func TestIndexMerge4PlanCache(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Len(t, res.Rows(), 6) - require.Regexp(t, ".*Selection.*", res.Rows()[0][0]) - require.Regexp(t, ".*IndexMerge.*", res.Rows()[1][0]) - require.Regexp(t, ".*TableRangeScan.*", res.Rows()[2][0]) - require.Equal(t, "range:(0,3), keep order:false, stats:pseudo", res.Rows()[2][4]) - require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[3][0]) - require.Equal(t, "range:(1,+inf], keep order:false, stats:pseudo", res.Rows()[3][4]) + require.Len(t, res.Rows(), 5) + require.Regexp(t, ".*IndexMerge.*", res.Rows()[0][0]) + require.Regexp(t, ".*TableRangeScan.*", res.Rows()[1][0]) + require.Equal(t, "range:(0,3), keep order:false, stats:pseudo", res.Rows()[1][4]) + require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[2][0]) + require.Equal(t, "range:(1,+inf], keep order:false, stats:pseudo", res.Rows()[2][4]) // test for prefix index tk.MustExec("drop table if exists t1;") @@ -821,16 +807,16 @@ func TestIndexMerge4PlanCache(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Regexp(t, ".*IndexMerge.*", res.Rows()[1][0]) + require.Regexp(t, ".*IndexMerge.*", res.Rows()[0][0]) tk.MustQuery("execute stmt using @b;").Check(testkit.Rows("3 ddcdsaf 3")) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) // unsafe range tk.MustQuery("execute stmt using @b;").Check(testkit.Rows("3 ddcdsaf 3")) tkProcess = tk.Session().ShowProcess() ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Regexp(t, ".*IndexMerge.*", res.Rows()[1][0]) + require.Regexp(t, ".*IndexMerge.*", res.Rows()[0][0]) // rewrite the origin indexMerge test tk.MustExec("drop table if exists t;") @@ -879,7 +865,7 @@ func TestIndexMerge4PlanCache(t *testing.T) { tk.MustExec("set @a=9, @b=10, @c=11;") tk.MustQuery("execute stmt using @c, @a;").Check(testkit.Rows("10 10 10")) tk.MustQuery("execute stmt using @a, @c;").Check(testkit.Rows("10 10 10", "11 11 11")) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) tk.MustQuery("execute stmt using @a, @a;").Check(testkit.Rows("10 10 10")) tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) diff --git a/executor/grant.go b/executor/grant.go index 3bae8e4956075..2933126cb855a 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -453,6 +453,9 @@ func (e *GrantExec) grantLevelPriv(priv *ast.PrivElem, user *ast.UserSpec, inter if priv.Priv == mysql.ExtendedPriv { return e.grantDynamicPriv(priv.Name, user, internalSession) } + if priv.Priv == mysql.UsagePriv { + return nil + } switch e.Level.Level { case ast.GrantLevelGlobal: return e.grantGlobalLevel(priv, user, internalSession) @@ -491,10 +494,6 @@ func (e *GrantExec) grantDynamicPriv(privName string, user *ast.UserSpec, intern // grantGlobalLevel manipulates mysql.user table. func (e *GrantExec) grantGlobalLevel(priv *ast.PrivElem, user *ast.UserSpec, internalSession sessionctx.Context) error { - if priv.Priv == 0 || priv.Priv == mysql.UsagePriv { - return nil - } - sql := new(strings.Builder) sqlexec.MustFormatSQL(sql, `UPDATE %n.%n SET `, mysql.SystemDB, mysql.UserTable) err := composeGlobalPrivUpdate(sql, priv.Priv, "Y") @@ -510,9 +509,6 @@ func (e *GrantExec) grantGlobalLevel(priv *ast.PrivElem, user *ast.UserSpec, int // grantDBLevel manipulates mysql.db table. func (e *GrantExec) grantDBLevel(priv *ast.PrivElem, user *ast.UserSpec, internalSession sessionctx.Context) error { - if priv.Priv == mysql.UsagePriv { - return nil - } for _, v := range mysql.StaticGlobalOnlyPrivs { if v == priv.Priv { return ErrWrongUsage.GenWithStackByArgs("DB GRANT", "GLOBAL PRIVILEGES") @@ -539,9 +535,6 @@ func (e *GrantExec) grantDBLevel(priv *ast.PrivElem, user *ast.UserSpec, interna // grantTableLevel manipulates mysql.tables_priv table. func (e *GrantExec) grantTableLevel(priv *ast.PrivElem, user *ast.UserSpec, internalSession sessionctx.Context) error { - if priv.Priv == mysql.UsagePriv { - return nil - } dbName := e.Level.DBName if len(dbName) == 0 { dbName = e.ctx.GetSessionVars().CurrentDB diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 44e36d2a26ae1..0906875337661 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1126,6 +1126,8 @@ func (e *memtableRetriever) setDataFromPartitions(ctx context.Context, sctx sess partitionMethod = "RANGE COLUMNS" case model.PartitionTypeList: partitionMethod = "LIST COLUMNS" + case model.PartitionTypeKey: + partitionMethod = "KEY" default: return fmt.Errorf("Inconsistent partition type, have type %v, but with COLUMNS > 0 (%d)", table.Partition.Type, len(table.Partition.Columns)) } diff --git a/executor/load_data.go b/executor/load_data.go index 5d206bd0149bf..2720d182d6b69 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/table" @@ -43,12 +44,13 @@ import ( "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" + "golang.org/x/sync/errgroup" ) const ( // LoadDataFormatSQLDump represents the data source file of LOAD DATA is // mydumper-format DML file - LoadDataFormatSQLDump = "sqldumpfile" + LoadDataFormatSQLDump = "sql file" // LoadDataFormatParquet represents the data source file of LOAD DATA is // parquet LoadDataFormatParquet = "parquet" @@ -64,59 +66,56 @@ var ( type LoadDataExec struct { baseExecutor - FileLocRef ast.FileLocRefTp - OnDuplicate ast.OnDuplicateKeyHandlingType - loadDataInfo *LoadDataInfo + FileLocRef ast.FileLocRefTp + OnDuplicate ast.OnDuplicateKeyHandlingType + loadDataWorker *LoadDataWorker } // Next implements the Executor Next interface. func (e *LoadDataExec) Next(ctx context.Context, req *chunk.Chunk) error { req.GrowAndReset(e.maxChunkSize) - if e.loadDataInfo.Path == "" { - return errors.New("Load Data: infile path is empty") - } - if !e.loadDataInfo.Table.Meta().IsBaseTable() { - return errors.New("can only load data into base tables") + if e.loadDataWorker.Path == "" { + return ErrLoadDataEmptyPath } // CSV-like - if e.loadDataInfo.Format == "" { - if e.loadDataInfo.NullInfo != nil && e.loadDataInfo.NullInfo.OptEnclosed && - (e.loadDataInfo.FieldsInfo == nil || e.loadDataInfo.FieldsInfo.Enclosed == nil) { - return errors.New("must specify FIELDS [OPTIONALLY] ENCLOSED BY when use NULL DEFINED BY OPTIONALLY ENCLOSED") + if e.loadDataWorker.format == "" { + if e.loadDataWorker.NullInfo != nil && e.loadDataWorker.NullInfo.OptEnclosed && + (e.loadDataWorker.FieldsInfo == nil || e.loadDataWorker.FieldsInfo.Enclosed == nil) { + return ErrLoadDataWrongFormatConfig.GenWithStackByArgs("must specify FIELDS [OPTIONALLY] ENCLOSED BY when use NULL DEFINED BY OPTIONALLY ENCLOSED") } // TODO: support lines terminated is "". - if len(e.loadDataInfo.LinesInfo.Terminated) == 0 { - return errors.New("Load Data: don't support load data terminated is nil") + if len(e.loadDataWorker.LinesInfo.Terminated) == 0 { + return ErrLoadDataWrongFormatConfig.GenWithStackByArgs("LINES TERMINATED BY is empty") } } switch e.FileLocRef { case ast.FileLocServerOrRemote: - u, err := storage.ParseRawURL(e.loadDataInfo.Path) + u, err := storage.ParseRawURL(e.loadDataWorker.Path) if err != nil { - return err + return ErrLoadDataInvalidURI.GenWithStackByArgs(err.Error()) } var filename string u.Path, filename = filepath.Split(u.Path) b, err := storage.ParseBackendFromURL(u, nil) if err != nil { - return err + return ErrLoadDataInvalidURI.GenWithStackByArgs(getMsgFromBRError(err)) } if b.GetLocal() != nil { - return errors.Errorf("Load Data: don't support load data from tidb-server's disk") + return ErrLoadDataFromServerDisk.GenWithStackByArgs(e.loadDataWorker.Path) } return e.loadFromRemote(ctx, b, filename) case ast.FileLocClient: // let caller use handleQuerySpecial to read data in this connection - sctx := e.loadDataInfo.ctx + sctx := e.loadDataWorker.ctx val := sctx.Value(LoadDataVarKey) if val != nil { sctx.SetValue(LoadDataVarKey, nil) - return errors.New("Load Data: previous load data option wasn't closed normally") + return errors.New("previous load data option wasn't closed normally") } - sctx.SetValue(LoadDataVarKey, e.loadDataInfo) + sctx.SetValue(LoadDataVarKey, e.loadDataWorker) } return nil } @@ -132,36 +131,36 @@ func (e *LoadDataExec) loadFromRemote( } s, err := storage.New(ctx, b, opt) if err != nil { - return err + return ErrLoadDataCantAccess } fileReader, err := s.Open(ctx, filename) if err != nil { - return err + return ErrLoadDataCantRead.GenWithStackByArgs(getMsgFromBRError(err), "Please check the INFILE path is correct") } defer fileReader.Close() - e.loadDataInfo.loadRemoteInfo = loadRemoteInfo{ + e.loadDataWorker.loadRemoteInfo = loadRemoteInfo{ store: s, path: filename, } - return e.loadDataInfo.Load(ctx, fileReader) + return e.loadDataWorker.Load(ctx, fileReader) } // Close implements the Executor Close interface. func (e *LoadDataExec) Close() error { - if e.runtimeStats != nil && e.loadDataInfo != nil && e.loadDataInfo.stats != nil { - defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.loadDataInfo.stats) + if e.runtimeStats != nil && e.loadDataWorker != nil && e.loadDataWorker.stats != nil { + defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.loadDataWorker.stats) } return nil } // Open implements the Executor Open interface. func (e *LoadDataExec) Open(ctx context.Context) error { - if e.loadDataInfo.insertColumns != nil { - e.loadDataInfo.initEvalBuffer() + if e.loadDataWorker.insertColumns != nil { + e.loadDataWorker.initEvalBuffer() } // Init for runtime stats. - e.loadDataInfo.collectRuntimeStatsEnabled() + e.loadDataWorker.collectRuntimeStatsEnabled() return nil } @@ -176,33 +175,69 @@ type loadRemoteInfo struct { path string } -// LoadDataInfo saves the information of loading data operation. -// TODO: rename it and remove unnecessary public methods. -type LoadDataInfo struct { +// LoadDataWorker does a LOAD DATA job. +type LoadDataWorker struct { *InsertValues - row []types.Datum - Path string - Format string - Table table.Table + Path string + Ctx sessionctx.Context + // expose some fields for test FieldsInfo *ast.FieldsClause LinesInfo *ast.LinesClause NullInfo *ast.NullDefinedBy IgnoreLines uint64 - Ctx sessionctx.Context - rows [][]types.Datum - Drained bool - ColumnAssignments []*ast.Assignment - ColumnsAndUserVars []*ast.ColumnNameOrUserVar - FieldMappings []*FieldMapping + format string + columnAssignments []*ast.Assignment + columnsAndUserVars []*ast.ColumnNameOrUserVar + fieldMappings []*FieldMapping + onDuplicate ast.OnDuplicateKeyHandlingType + table table.Table + row []types.Datum + rows [][]types.Datum commitTaskQueue chan commitTask - StopCh chan struct{} - QuitCh chan struct{} - OnDuplicate ast.OnDuplicateKeyHandlingType - - loadRemoteInfo loadRemoteInfo + loadRemoteInfo loadRemoteInfo +} + +// NewLoadDataWorker creates a new LoadDataWorker that is ready to work. +func NewLoadDataWorker( + sctx sessionctx.Context, + plan *plannercore.LoadData, + tbl table.Table, +) (*LoadDataWorker, error) { + insertVal := &InsertValues{ + baseExecutor: newBaseExecutor(sctx, nil, plan.ID()), + Table: tbl, + Columns: plan.Columns, + GenExprs: plan.GenCols.Exprs, + isLoadData: true, + txnInUse: sync.Mutex{}, + maxRowsInBatch: uint64(sctx.GetSessionVars().DMLBatchSize), + } + loadDataWorker := &LoadDataWorker{ + row: make([]types.Datum, 0, len(insertVal.insertColumns)), + commitTaskQueue: make(chan commitTask, taskQueueSize), + InsertValues: insertVal, + Path: plan.Path, + format: plan.Format, + table: tbl, + FieldsInfo: plan.FieldsInfo, + LinesInfo: plan.LinesInfo, + NullInfo: plan.NullInfo, + IgnoreLines: plan.IgnoreLines, + columnAssignments: plan.ColumnAssignments, + columnsAndUserVars: plan.ColumnsAndUserVars, + onDuplicate: plan.OnDuplicate, + Ctx: sctx, + } + columnNames := loadDataWorker.initFieldMappings() + err := loadDataWorker.initLoadColumns(columnNames) + if err != nil { + return nil, err + } + loadDataWorker.ResetBatch() + return loadDataWorker, nil } // FieldMapping indicates the relationship between input field and table column or user variable @@ -211,152 +246,59 @@ type FieldMapping struct { UserVar *ast.VariableExpr } -// Load reads from readerFn and do load data job. -func (e *LoadDataInfo) Load(ctx context.Context, reader io.ReadSeekCloser) error { - e.initQueues() - e.SetMaxRowsInBatch(uint64(e.Ctx.GetSessionVars().DMLBatchSize)) - e.startStopWatcher() - // let stop watcher goroutine quit - defer e.forceQuit() - err := sessiontxn.NewTxn(ctx, e.Ctx) - if err != nil { - return err - } - // processStream process input data, enqueue commit task - wg := new(sync.WaitGroup) - wg.Add(1) - go processStream(ctx, reader, e, wg) - err = e.commitWork(ctx) - wg.Wait() - return err -} +// initFieldMappings make a field mapping slice to implicitly map input field to table column or user defined variable +// the slice's order is the same as the order of the input fields. +// Returns a slice of same ordered column names without user defined variable names. +func (e *LoadDataWorker) initFieldMappings() []string { + columns := make([]string, 0, len(e.columnsAndUserVars)+len(e.columnAssignments)) + tableCols := e.table.Cols() -// processStream process input stream from network -func processStream(ctx context.Context, reader io.ReadSeekCloser, loadDataInfo *LoadDataInfo, wg *sync.WaitGroup) { - var ( - parser mydump.Parser - err error - ) - defer func() { - r := recover() - if r != nil { - logutil.Logger(ctx).Error("process routine panicked", - zap.Reflect("r", r), - zap.Stack("stack")) - } - if err != nil { - logutil.Logger(ctx).Error("process routine meet error", - zap.Error(err)) - } - if err != nil || r != nil { - loadDataInfo.forceQuit() - } else { - loadDataInfo.CloseTaskQueue() + if len(e.columnsAndUserVars) == 0 { + for _, v := range tableCols { + fieldMapping := &FieldMapping{ + Column: v, + } + e.fieldMappings = append(e.fieldMappings, fieldMapping) + columns = append(columns, v.Name.O) } - wg.Done() - }() - switch strings.ToLower(loadDataInfo.Format) { - case "": - // CSV-like - parser, err = mydump.NewCSVParser( - ctx, - loadDataInfo.GenerateCSVConfig(), - reader, - int64(config.ReadBlockSize), - nil, - false, - // TODO: support charset conversion - nil) - case LoadDataFormatSQLDump: - parser = mydump.NewChunkParser( - ctx, - loadDataInfo.Ctx.GetSessionVars().SQLMode, - reader, - int64(config.ReadBlockSize), - nil, - ) - case LoadDataFormatParquet: - if loadDataInfo.loadRemoteInfo.store == nil { - err = errors.New("parquet format requires remote storage") - return - } - parser, err = mydump.NewParquetParser( - ctx, - loadDataInfo.loadRemoteInfo.store, - reader, - loadDataInfo.loadRemoteInfo.path, - ) - default: - err = errors.Errorf("unsupported format: %s", loadDataInfo.Format) - } - if err != nil { - return + return columns } - parser.SetLogger(log.Logger{Logger: logutil.Logger(ctx)}) + var column *table.Column - for { - // prepare batch and enqueue task - err = loadDataInfo.ReadRows(ctx, parser) - if err != nil { - logutil.Logger(ctx).Error("load data process stream error in ReadRows", zap.Error(err)) - return - } - if loadDataInfo.curBatchCnt == 0 { - return - } - if err = loadDataInfo.enqOneTask(ctx); err != nil { - logutil.Logger(ctx).Error("load data process stream error in enqOneTask", zap.Error(err)) - return + for _, v := range e.columnsAndUserVars { + if v.ColumnName != nil { + column = table.FindCol(tableCols, v.ColumnName.Name.O) + columns = append(columns, v.ColumnName.Name.O) + } else { + column = nil } - } -} -// reorderColumns reorder the e.insertColumns according to the order of columnNames -// Note: We must ensure there must be one-to-one mapping between e.insertColumns and columnNames in terms of column name. -func (e *LoadDataInfo) reorderColumns(columnNames []string) error { - cols := e.insertColumns - - if len(cols) != len(columnNames) { - return ErrColumnsNotMatched - } - - reorderedColumns := make([]*table.Column, len(cols)) - - if columnNames == nil { - return nil - } - - mapping := make(map[string]int) - for idx, colName := range columnNames { - mapping[strings.ToLower(colName)] = idx - } - - for _, col := range cols { - idx := mapping[col.Name.L] - reorderedColumns[idx] = col + fieldMapping := &FieldMapping{ + Column: column, + UserVar: v.UserVar, + } + e.fieldMappings = append(e.fieldMappings, fieldMapping) } - e.insertColumns = reorderedColumns - - return nil + return columns } // initLoadColumns sets columns which the input fields loaded to. -func (e *LoadDataInfo) initLoadColumns(columnNames []string) error { +func (e *LoadDataWorker) initLoadColumns(columnNames []string) error { var cols []*table.Column var missingColName string var err error - tableCols := e.Table.Cols() + tableCols := e.table.Cols() if len(columnNames) != len(tableCols) { - for _, v := range e.ColumnAssignments { + for _, v := range e.columnAssignments { columnNames = append(columnNames, v.Column.Name.O) } } - cols, missingColName = table.FindCols(tableCols, columnNames, e.Table.Meta().PKIsHandle) + cols, missingColName = table.FindCols(tableCols, columnNames, e.table.Meta().PKIsHandle) if missingColName != "" { return dbterror.ErrBadField.GenWithStackByArgs(missingColName, "field list") } @@ -390,104 +332,189 @@ func (e *LoadDataInfo) initLoadColumns(columnNames []string) error { return nil } -// initFieldMappings make a field mapping slice to implicitly map input field to table column or user defined variable -// the slice's order is the same as the order of the input fields. -// Returns a slice of same ordered column names without user defined variable names. -func (e *LoadDataInfo) initFieldMappings() []string { - columns := make([]string, 0, len(e.ColumnsAndUserVars)+len(e.ColumnAssignments)) - tableCols := e.Table.Cols() - - if len(e.ColumnsAndUserVars) == 0 { - for _, v := range tableCols { - fieldMapping := &FieldMapping{ - Column: v, - } - e.FieldMappings = append(e.FieldMappings, fieldMapping) - columns = append(columns, v.Name.O) - } +// reorderColumns reorder the e.insertColumns according to the order of columnNames +// Note: We must ensure there must be one-to-one mapping between e.insertColumns and columnNames in terms of column name. +func (e *LoadDataWorker) reorderColumns(columnNames []string) error { + cols := e.insertColumns - return columns + if len(cols) != len(columnNames) { + return ErrColumnsNotMatched } - var column *table.Column + reorderedColumns := make([]*table.Column, len(cols)) - for _, v := range e.ColumnsAndUserVars { - if v.ColumnName != nil { - column = table.FindCol(tableCols, v.ColumnName.Name.O) - columns = append(columns, v.ColumnName.Name.O) - } else { - column = nil - } + if columnNames == nil { + return nil + } - fieldMapping := &FieldMapping{ - Column: column, - UserVar: v.UserVar, - } - e.FieldMappings = append(e.FieldMappings, fieldMapping) + mapping := make(map[string]int) + for idx, colName := range columnNames { + mapping[strings.ToLower(colName)] = idx } - return columns -} + for _, col := range cols { + idx := mapping[col.Name.L] + reorderedColumns[idx] = col + } -// GetRows getter for rows -func (e *LoadDataInfo) GetRows() [][]types.Datum { - return e.rows -} + e.insertColumns = reorderedColumns -// GetCurBatchCnt getter for curBatchCnt -func (e *LoadDataInfo) GetCurBatchCnt() uint64 { - return e.curBatchCnt + return nil } -// CloseTaskQueue preparing routine to inform commit routine no more data -func (e *LoadDataInfo) CloseTaskQueue() { - close(e.commitTaskQueue) -} +// Load reads from readerFn and do load data job. +func (e *LoadDataWorker) Load(ctx context.Context, reader io.ReadSeekCloser) error { + var ( + parser mydump.Parser + err error + ) -// initQueues initialize task queue and error report queue -func (e *LoadDataInfo) initQueues() { - e.commitTaskQueue = make(chan commitTask, taskQueueSize) - e.StopCh = make(chan struct{}, 2) - e.QuitCh = make(chan struct{}) + switch strings.ToLower(e.format) { + case "": + // CSV-like + parser, err = mydump.NewCSVParser( + ctx, + e.GenerateCSVConfig(), + reader, + int64(config.ReadBlockSize), + nil, + false, + // TODO: support charset conversion + nil) + case LoadDataFormatSQLDump: + parser = mydump.NewChunkParser( + ctx, + e.Ctx.GetSessionVars().SQLMode, + reader, + int64(config.ReadBlockSize), + nil, + ) + case LoadDataFormatParquet: + if e.loadRemoteInfo.store == nil { + return ErrLoadParquetFromLocal + } + parser, err = mydump.NewParquetParser( + ctx, + e.loadRemoteInfo.store, + reader, + e.loadRemoteInfo.path, + ) + default: + return ErrLoadDataUnsupportedFormat.GenWithStackByArgs(e.format) + } + if err != nil { + return ErrLoadDataWrongFormatConfig.GenWithStack(err.Error()) + } + parser.SetLogger(log.Logger{Logger: logutil.Logger(ctx)}) + + err = sessiontxn.NewTxn(ctx, e.Ctx) + if err != nil { + return err + } + group, groupCtx := errgroup.WithContext(ctx) + + // processStream process input data, enqueue commit task + group.Go(func() error { + return e.processStream(groupCtx, parser) + }) + group.Go(func() error { + return e.commitWork(groupCtx) + }) + + err = group.Wait() + e.SetMessage() + return err } -// startStopWatcher monitor StopCh to force quit -func (e *LoadDataInfo) startStopWatcher() { - go func() { - <-e.StopCh - close(e.QuitCh) +// processStream process input stream from network +func (e *LoadDataWorker) processStream( + ctx context.Context, + parser mydump.Parser, +) (err error) { + defer func() { + r := recover() + if r != nil { + logutil.Logger(ctx).Error("process routine panicked", + zap.Reflect("r", r), + zap.Stack("stack")) + err = errors.Errorf("%v", r) + } }() -} -// forceQuit let commit quit directly -func (e *LoadDataInfo) forceQuit() { - e.StopCh <- struct{}{} + checkKilled := time.NewTicker(30 * time.Second) + defer checkKilled.Stop() + + for { + // prepare batch and enqueue task + if err = e.ReadRows(ctx, parser); err != nil { + return + } + if e.curBatchCnt == 0 { + close(e.commitTaskQueue) + return + } + + TrySendTask: + select { + case <-ctx.Done(): + return ctx.Err() + case <-checkKilled.C: + if atomic.CompareAndSwapUint32(&e.Ctx.GetSessionVars().Killed, 1, 0) { + logutil.Logger(ctx).Info("load data query interrupted quit data processing") + close(e.commitTaskQueue) + return ErrQueryInterrupted + } + goto TrySendTask + case e.commitTaskQueue <- commitTask{e.curBatchCnt, e.rows}: + } + // reset rows buffer, will reallocate buffer but NOT reuse + e.ResetBatch() + } } -// makeCommitTask produce commit task with data in LoadDataInfo.rows LoadDataInfo.curBatchCnt -func (e *LoadDataInfo) makeCommitTask() commitTask { - return commitTask{e.curBatchCnt, e.rows} +// ResetBatch reset the inner batch. +func (e *LoadDataWorker) ResetBatch() { + e.rows = make([][]types.Datum, 0, e.maxRowsInBatch) + e.curBatchCnt = 0 } -// enqOneTask feed one batch commit task to commit work -func (e *LoadDataInfo) enqOneTask(ctx context.Context) error { - var err error - if e.curBatchCnt > 0 { +// commitWork commit batch sequentially +func (e *LoadDataWorker) commitWork(ctx context.Context) (err error) { + defer func() { + r := recover() + if r != nil { + logutil.Logger(ctx).Error("commitWork panicked", + zap.Reflect("r", r), + zap.Stack("stack")) + err = errors.Errorf("%v", r) + } + }() + + var tasks uint64 + for { select { - case e.commitTaskQueue <- e.makeCommitTask(): - case <-e.QuitCh: - err = errors.New("enqOneTask forced to quit") - logutil.Logger(ctx).Error("enqOneTask forced to quit, possible commitWork error") - return err + case <-ctx.Done(): + return ctx.Err() + case task, ok := <-e.commitTaskQueue: + if !ok { + return nil + } + start := time.Now() + if err = e.commitOneTask(ctx, task); err != nil { + return err + } + tasks++ + logutil.Logger(ctx).Info("commit one task success", + zap.Duration("commit time usage", time.Since(start)), + zap.Uint64("keys processed", task.cnt), + zap.Uint64("tasks processed", tasks), + zap.Int("tasks in queue", len(e.commitTaskQueue))) } - // reset rows buffer, will reallocate buffer but NOT reuse - e.SetMaxRowsInBatch(e.maxRowsInBatch) } - return err } -// CommitOneTask insert Data from LoadDataInfo.rows, then make commit and refresh txn -func (e *LoadDataInfo) CommitOneTask(ctx context.Context, task commitTask) error { +// commitOneTask insert Data from LoadDataWorker.rows, then make commit and refresh txn +func (e *LoadDataWorker) commitOneTask(ctx context.Context, task commitTask) error { var err error defer func() { if err != nil { @@ -514,66 +541,44 @@ func (e *LoadDataInfo) CommitOneTask(ctx context.Context, task commitTask) error return err } -// commitWork commit batch sequentially -func (e *LoadDataInfo) commitWork(ctx context.Context) error { +// CheckAndInsertOneBatch is used to commit one transaction batch fulfilled data +func (e *LoadDataWorker) CheckAndInsertOneBatch(ctx context.Context, rows [][]types.Datum, cnt uint64) error { + if e.stats != nil && e.stats.BasicRuntimeStats != nil { + // Since this method will not call by executor Next, + // so we need record the basic executor runtime stats by ourselves. + start := time.Now() + defer func() { + e.stats.BasicRuntimeStats.Record(time.Since(start), 0) + }() + } var err error - defer func() { - r := recover() - if r != nil { - logutil.Logger(ctx).Error("commitWork panicked", - zap.Reflect("r", r), - zap.Stack("stack")) - } - if err != nil || r != nil { - e.forceQuit() - } - if err != nil { - e.ctx.StmtRollback(ctx, false) - } - }() - var tasks uint64 - var end = false - for !end { - select { - case <-e.QuitCh: - err = errors.New("commit forced to quit") - logutil.Logger(ctx).Error("commit forced to quit, possible preparation failed") - return err - case commitTask, ok := <-e.commitTaskQueue: - if ok { - start := time.Now() - err = e.CommitOneTask(ctx, commitTask) - if err != nil { - break - } - tasks++ - logutil.Logger(ctx).Info("commit one task success", - zap.Duration("commit time usage", time.Since(start)), - zap.Uint64("keys processed", commitTask.cnt), - zap.Uint64("tasks processed", tasks), - zap.Int("tasks in queue", len(e.commitTaskQueue))) - } else { - end = true - } - } - if err != nil { - logutil.Logger(ctx).Error("load data commit work error", zap.Error(err)) - break - } - if atomic.CompareAndSwapUint32(&e.Ctx.GetSessionVars().Killed, 1, 0) { - logutil.Logger(ctx).Info("load data query interrupted quit data processing") - err = ErrQueryInterrupted - break - } + if cnt == 0 { + return err + } + e.ctx.GetSessionVars().StmtCtx.AddRecordRows(cnt) + + replace := false + if e.onDuplicate == ast.OnDuplicateKeyHandlingReplace { + replace = true + } + + err = e.batchCheckAndInsert(ctx, rows[0:cnt], e.addRecordLD, replace) + if err != nil { + return err } return err } -// SetMaxRowsInBatch sets the max number of rows to insert in a batch. -func (e *LoadDataInfo) SetMaxRowsInBatch(limit uint64) { - e.maxRowsInBatch = limit - e.rows = make([][]types.Datum, 0, limit) - e.curBatchCnt = 0 +func (e *LoadDataWorker) addRecordLD(ctx context.Context, row []types.Datum) error { + if row == nil { + return nil + } + err := e.addRecord(ctx, row) + if err != nil { + e.handleWarning(err) + return err + } + return nil } // ReadRows reads rows from parser. When parser's reader meet EOF, it will return @@ -581,7 +586,7 @@ func (e *LoadDataInfo) SetMaxRowsInBatch(limit uint64) { // will also return nil. // The result rows are saved in e.rows and update some members, caller can check // if curBatchCnt == 0 to know if reached EOF. -func (e *LoadDataInfo) ReadRows(ctx context.Context, parser mydump.Parser) error { +func (e *LoadDataWorker) ReadRows(ctx context.Context, parser mydump.Parser) error { ignoreOneLineFn := parser.ReadRow if csvParser, ok := parser.(*mydump.CSVParser); ok { ignoreOneLineFn = func() error { @@ -606,7 +611,10 @@ func (e *LoadDataInfo) ReadRows(ctx context.Context, parser mydump.Parser) error if errors.Cause(err) == io.EOF { return nil } - return err + return ErrLoadDataCantRead.GenWithStackByArgs( + err.Error(), + "Only the following formats delimited text file (csv, tsv), parquet, sql are supported. Please provide the valid source file(s)", + ) } // rowCount will be used in fillRow(), last insert ID will be assigned according to the rowCount = 1. // So should add first here. @@ -621,48 +629,8 @@ func (e *LoadDataInfo) ReadRows(ctx context.Context, parser mydump.Parser) error } } -// CheckAndInsertOneBatch is used to commit one transaction batch full filled data -func (e *LoadDataInfo) CheckAndInsertOneBatch(ctx context.Context, rows [][]types.Datum, cnt uint64) error { - if e.stats != nil && e.stats.BasicRuntimeStats != nil { - // Since this method will not call by executor Next, - // so we need record the basic executor runtime stats by ourself. - start := time.Now() - defer func() { - e.stats.BasicRuntimeStats.Record(time.Since(start), 0) - }() - } - var err error - if cnt == 0 { - return err - } - e.ctx.GetSessionVars().StmtCtx.AddRecordRows(cnt) - - replace := false - if e.OnDuplicate == ast.OnDuplicateKeyHandlingReplace { - replace = true - } - - err = e.batchCheckAndInsert(ctx, rows[0:cnt], e.addRecordLD, replace) - if err != nil { - return err - } - return err -} - -// SetMessage sets info message(ERR_LOAD_INFO) generated by LOAD statement, it is public because of the special way that -// LOAD statement is handled. -func (e *LoadDataInfo) SetMessage() { - stmtCtx := e.ctx.GetSessionVars().StmtCtx - numRecords := stmtCtx.RecordRows() - numDeletes := stmtCtx.DeletedRows() - numSkipped := numRecords - stmtCtx.CopiedRows() - numWarnings := stmtCtx.WarningCount() - msg := fmt.Sprintf(mysql.MySQLErrName[mysql.ErrLoadInfo].Raw, numRecords, numDeletes, numSkipped, numWarnings) - e.ctx.GetSessionVars().StmtCtx.SetMessage(msg) -} - // colsToRow encodes the data of parser output. -func (e *LoadDataInfo) colsToRow(ctx context.Context, cols []types.Datum) []types.Datum { +func (e *LoadDataWorker) colsToRow(ctx context.Context, cols []types.Datum) []types.Datum { row := make([]types.Datum, 0, len(e.insertColumns)) sessionVars := e.Ctx.GetSessionVars() setVar := func(name string, col *types.Datum) { @@ -676,16 +644,16 @@ func (e *LoadDataInfo) colsToRow(ctx context.Context, cols []types.Datum) []type } } - for i := 0; i < len(e.FieldMappings); i++ { + for i := 0; i < len(e.fieldMappings); i++ { if i >= len(cols) { - if e.FieldMappings[i].Column == nil { - setVar(e.FieldMappings[i].UserVar.Name, nil) + if e.fieldMappings[i].Column == nil { + setVar(e.fieldMappings[i].UserVar.Name, nil) continue } // If some columns is missing and their type is time and has not null flag, they should be set as current time. - if types.IsTypeTime(e.FieldMappings[i].Column.GetType()) && mysql.HasNotNullFlag(e.FieldMappings[i].Column.GetFlag()) { - row = append(row, types.NewTimeDatum(types.CurrentTime(e.FieldMappings[i].Column.GetType()))) + if types.IsTypeTime(e.fieldMappings[i].Column.GetType()) && mysql.HasNotNullFlag(e.fieldMappings[i].Column.GetFlag()) { + row = append(row, types.NewTimeDatum(types.CurrentTime(e.fieldMappings[i].Column.GetType()))) continue } @@ -693,8 +661,8 @@ func (e *LoadDataInfo) colsToRow(ctx context.Context, cols []types.Datum) []type continue } - if e.FieldMappings[i].Column == nil { - setVar(e.FieldMappings[i].UserVar.Name, &cols[i]) + if e.fieldMappings[i].Column == nil { + setVar(e.fieldMappings[i].UserVar.Name, &cols[i]) continue } @@ -705,9 +673,9 @@ func (e *LoadDataInfo) colsToRow(ctx context.Context, cols []types.Datum) []type row = append(row, cols[i]) } - for i := 0; i < len(e.ColumnAssignments); i++ { + for i := 0; i < len(e.columnAssignments); i++ { // eval expression of `SET` clause - d, err := expression.EvalAstExpr(e.Ctx, e.ColumnAssignments[i].Expr) + d, err := expression.EvalAstExpr(e.Ctx, e.columnAssignments[i].Expr) if err != nil { e.handleWarning(err) return nil @@ -725,20 +693,20 @@ func (e *LoadDataInfo) colsToRow(ctx context.Context, cols []types.Datum) []type return newRow } -func (e *LoadDataInfo) addRecordLD(ctx context.Context, row []types.Datum) error { - if row == nil { - return nil - } - err := e.addRecord(ctx, row) - if err != nil { - e.handleWarning(err) - return err - } - return nil +// SetMessage sets info message(ERR_LOAD_INFO) generated by LOAD statement, it is public because of the special way that +// LOAD statement is handled. +func (e *LoadDataWorker) SetMessage() { + stmtCtx := e.ctx.GetSessionVars().StmtCtx + numRecords := stmtCtx.RecordRows() + numDeletes := stmtCtx.DeletedRows() + numSkipped := numRecords - stmtCtx.CopiedRows() + numWarnings := stmtCtx.WarningCount() + msg := fmt.Sprintf(mysql.MySQLErrName[mysql.ErrLoadInfo].Raw, numRecords, numDeletes, numSkipped, numWarnings) + e.ctx.GetSessionVars().StmtCtx.SetMessage(msg) } -// GenerateCSVConfig generates a CSV config for parser from LoadDataInfo. -func (e *LoadDataInfo) GenerateCSVConfig() *config.CSVConfig { +// GenerateCSVConfig generates a CSV config for parser from LoadDataWorker. +func (e *LoadDataWorker) GenerateCSVConfig() *config.CSVConfig { var ( nullDef []string quotedNullIsText = true @@ -780,6 +748,16 @@ func (e *LoadDataInfo) GenerateCSVConfig() *config.CSVConfig { } } +// GetRows getter for rows +func (e *LoadDataWorker) GetRows() [][]types.Datum { + return e.rows +} + +// GetCurBatchCnt getter for curBatchCnt +func (e *LoadDataWorker) GetCurBatchCnt() uint64 { + return e.curBatchCnt +} + var _ io.ReadSeekCloser = (*SimpleSeekerOnReadCloser)(nil) // SimpleSeekerOnReadCloser provides Seek(0, SeekCurrent) on ReadCloser. diff --git a/executor/loadremotetest/BUILD.bazel b/executor/loadremotetest/BUILD.bazel index 57ddea3ad19ea..aa5279fac652c 100644 --- a/executor/loadremotetest/BUILD.bazel +++ b/executor/loadremotetest/BUILD.bazel @@ -4,6 +4,7 @@ go_test( name = "loadremotetest_test", timeout = "short", srcs = [ + "error_test.go", "main_test.go", "one_csv_test.go", "one_parquet_test.go", @@ -15,8 +16,11 @@ go_test( deps = [ "//executor", "//kv", + "//parser/terror", "//testkit", "@com_github_fsouza_fake_gcs_server//fakestorage", + "@com_github_pingcap_errors//:errors", + "@com_github_stretchr_testify//require", "@com_github_stretchr_testify//suite", "@org_uber_go_goleak//:goleak", ], diff --git a/executor/loadremotetest/error_test.go b/executor/loadremotetest/error_test.go new file mode 100644 index 0000000000000..c47ae6f6764be --- /dev/null +++ b/executor/loadremotetest/error_test.go @@ -0,0 +1,117 @@ +// 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 loadremotetest + +import ( + "fmt" + "testing" + + "github.com/fsouza/fake-gcs-server/fakestorage" + "github.com/pingcap/errors" + "github.com/pingcap/tidb/parser/terror" + "github.com/stretchr/testify/require" +) + +func checkClientErrorMessage(t *testing.T, err error, msg string) { + require.Error(t, err) + cause := errors.Cause(err) + terr, ok := cause.(*errors.Error) + require.True(t, ok, "%T", cause) + require.Contains(t, terror.ToSQLError(terr).Error(), msg) +} + +func (s *mockGCSSuite) TestErrorMessage() { + s.tk.MustExec("DROP DATABASE IF EXISTS load_csv;") + + err := s.tk.ExecToErr("LOAD DATA INFILE 'gs://1' INTO TABLE t") + checkClientErrorMessage(s.T(), err, "ERROR 1046 (3D000): No database selected") + err = s.tk.ExecToErr("LOAD DATA INFILE 'gs://1' INTO TABLE wrongdb.t") + checkClientErrorMessage(s.T(), err, "ERROR 1146 (42S02): Table 'wrongdb.t' doesn't exist") + + s.tk.MustExec("CREATE DATABASE load_csv;") + s.tk.MustExec("USE load_csv;") + s.tk.MustExec("CREATE TABLE t (i INT PRIMARY KEY, s varchar(32));") + + err = s.tk.ExecToErr("LOAD DATA INFILE 'gs://1' INTO TABLE t (wrong)") + checkClientErrorMessage(s.T(), err, "ERROR 1054 (42S22): Unknown column 'wrong' in 'field list'") + // This behaviour is different from MySQL + err = s.tk.ExecToErr("LOAD DATA INFILE 'gs://1' INTO TABLE t (i,i)") + checkClientErrorMessage(s.T(), err, "ERROR 1110 (42000): Column 'i' specified twice") + err = s.tk.ExecToErr("LOAD DATA INFILE 'gs://1' INTO TABLE t (@v) SET wrong=@v") + checkClientErrorMessage(s.T(), err, "ERROR 1054 (42S22): Unknown column 'wrong' in 'field list'") + err = s.tk.ExecToErr("LOAD DATA INFILE 'abc://1' INTO TABLE t;") + checkClientErrorMessage(s.T(), err, + "ERROR 8158 (HY000): The URI of INFILE is invalid. Reason: storage abc not support yet. Please provide a valid URI, such as 's3://import/test.csv?access_key_id={your_access_key_id ID}&secret_access_key={your_secret_access_key}&session_token={your_session_token}'") + err = s.tk.ExecToErr("LOAD DATA INFILE 's3://no-network' INTO TABLE t;") + checkClientErrorMessage(s.T(), err, + "ERROR 8159 (HY000): Access to the source file has been denied. Please check the URI, access key and secret access key are correct") + err = s.tk.ExecToErr(fmt.Sprintf(`LOAD DATA INFILE 'gs://wrong-bucket/p?endpoint=%s' + INTO TABLE t;`, gcsEndpoint)) + checkClientErrorMessage(s.T(), err, + "ERROR 8160 (HY000): Failed to read source files. Reason: failed to read gcs file, file info: input.bucket='wrong-bucket', input.key='p'. Please check the INFILE path is correct") + + s.server.CreateObject(fakestorage.Object{ + ObjectAttrs: fakestorage.ObjectAttrs{ + BucketName: "test-tsv", + Name: "t.tsv", + }, + Content: []byte("1\t2\n" + + "1\t4\n"), + }) + err = s.tk.ExecToErr(fmt.Sprintf(`LOAD DATA INFILE 'gs://test-tsv/t.tsv?endpoint=%s' + FORMAT '123' INTO TABLE t;`, gcsEndpoint)) + checkClientErrorMessage(s.T(), err, + "ERROR 8157 (HY000): The FORMAT '123' is not supported") + err = s.tk.ExecToErr(fmt.Sprintf(`LOAD DATA INFILE 'gs://test-tsv/t.tsv?endpoint=%s' + FORMAT 'sql file' INTO TABLE t;`, gcsEndpoint)) + checkClientErrorMessage(s.T(), err, + "ERROR 8160 (HY000): Failed to read source files. Reason: syntax error: unexpected Integer (1) at offset 1, expecting start of row. Only the following formats delimited text file (csv, tsv), parquet, sql are supported. Please provide the valid source file(s)") + err = s.tk.ExecToErr(fmt.Sprintf(`LOAD DATA INFILE 'gs://test-tsv/t.tsv?endpoint=%s' + INTO TABLE t LINES STARTING BY '\n';`, gcsEndpoint)) + checkClientErrorMessage(s.T(), err, + `ERROR 8162 (HY000): STARTING BY ' +' cannot contain LINES TERMINATED BY ' +'`) + + // TODO: fix these tests + //s.tk.MustExec("CREATE TABLE t2 (c1 INT, c2 INT, c3 INT);") + //err = s.tk.ExecToErr(fmt.Sprintf(`LOAD DATA INFILE 'gs://test-tsv/t.tsv?endpoint=%s' + // INTO TABLE t2;`, gcsEndpoint)) + //checkClientErrorMessage(s.T(), err, + // "ERROR 1261 (01000): Row 1 doesn't contain data for all columns") + //s.tk.MustExec("CREATE TABLE t3 (c1 INT);") + //err = s.tk.ExecToErr(fmt.Sprintf(`LOAD DATA INFILE 'gs://test-tsv/t.tsv?endpoint=%s' + // INTO TABLE t3;`, gcsEndpoint)) + //checkClientErrorMessage(s.T(), err, + // "ERROR 1262 (01000): Row 1 was truncated; it contained more data than there were input columns") + + // TODO: don't use batchCheckAndInsert, mimic (*InsertExec).exec() + + //err = s.tk.ExecToErr(fmt.Sprintf(`LOAD DATA INFILE 'gs://test-tsv/t.tsv?endpoint=%s' + // INTO TABLE t;`, gcsEndpoint)) + //checkClientErrorMessage(s.T(), err, "ERROR 1062 (23000): Duplicate entry '1' for key 'PRIMARY'") + + //s.server.CreateObject(fakestorage.Object{ + // ObjectAttrs: fakestorage.ObjectAttrs{ + // BucketName: "test-tsv", + // Name: "t2.tsv", + // }, + // Content: []byte("null\t2\n" + + // "3\t4\n"), + //}) + //err = s.tk.ExecToErr(fmt.Sprintf(`LOAD DATA INFILE 'gs://test-tsv/t2.tsv?endpoint=%s' + // INTO TABLE t NULL DEFINED BY 'null';`, gcsEndpoint)) + //checkClientErrorMessage(s.T(), err, "ERROR 8154 (HY000): LOAD DATA raises error(s): xxx") +} diff --git a/executor/loadremotetest/one_csv_test.go b/executor/loadremotetest/one_csv_test.go index ff9fd277f7dfe..cc98e6ff68af8 100644 --- a/executor/loadremotetest/one_csv_test.go +++ b/executor/loadremotetest/one_csv_test.go @@ -81,7 +81,7 @@ func (s *mockGCSSuite) TestLoadCSV() { // can't read file at tidb-server sql = "LOAD DATA INFILE '/etc/passwd' INTO TABLE load_csv.t;" - s.tk.MustContainErrMsg(sql, "don't support load data from tidb-server") + s.tk.MustContainErrMsg(sql, "Don't support load data from tidb-server's disk. Or if you want to load local data via client, the path of INFILE '/etc/passwd' needs to specify the clause of LOCAL first") } func (s *mockGCSSuite) TestIgnoreNLines() { diff --git a/executor/loadremotetest/one_sqldump_test.go b/executor/loadremotetest/one_sqldump_test.go index ec320787e3b05..e2755c8f0d74b 100644 --- a/executor/loadremotetest/one_sqldump_test.go +++ b/executor/loadremotetest/one_sqldump_test.go @@ -36,7 +36,7 @@ func (s *mockGCSSuite) TestLoadSQLDump() { }) sql := fmt.Sprintf(`LOAD DATA INFILE 'gs://test-load-parquet/p?endpoint=%s' - FORMAT 'SQLDumpFile' INTO TABLE load_csv.t;`, gcsEndpoint) + FORMAT 'SQL file' INTO TABLE load_csv.t;`, gcsEndpoint) s.tk.MustExec(sql) s.tk.MustQuery("SELECT * FROM load_csv.t;").Check(testkit.Rows( "1 a", diff --git a/executor/prepared_test.go b/executor/prepared_test.go index 6b0b6ab10c7ef..d725509523386 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -171,9 +171,8 @@ func TestIssue29850(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // cannot use PointGet since it contains a range condition - `Selection_7 1.00 root ge(test.t.a, 1), le(test.t.a, 1)`, - `└─TableReader_6 1.00 root data:TableRangeScan_5`, - ` └─TableRangeScan_5 1.00 cop[tikv] table:t range:[1,1], keep order:false, stats:pseudo`)) + `TableReader_6 1.00 root data:TableRangeScan_5`, + `└─TableRangeScan_5 1.00 cop[tikv] table:t range:[1,1], keep order:false, stats:pseudo`)) tk.MustQuery(`execute stmt using @a1, @a2`).Check(testkit.Rows("1", "2")) tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) @@ -183,9 +182,8 @@ func TestIssue29850(t *testing.T) { ps = []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // cannot use PointGet since it contains a or condition - `Selection_7 1.00 root or(eq(test.t.a, 1), eq(test.t.a, 1))`, - `└─TableReader_6 1.00 root data:TableRangeScan_5`, - ` └─TableRangeScan_5 1.00 cop[tikv] table:t range:[1,1], keep order:false, stats:pseudo`)) + `TableReader_6 1.00 root data:TableRangeScan_5`, + `└─TableRangeScan_5 1.00 cop[tikv] table:t range:[1,1], keep order:false, stats:pseudo`)) tk.MustQuery(`execute stmt using @a1, @a2`).Check(testkit.Rows("1", "2")) } @@ -210,10 +208,10 @@ func TestIssue28064(t *testing.T) { ps := []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) - rows.Check(testkit.Rows("Selection_8 0.00 root eq(test.t28064.a, 123), eq(test.t28064.b, 234), eq(test.t28064.c, 345)", - "└─IndexLookUp_7 0.00 root ", - " ├─IndexRangeScan_5(Build) 0.00 cop[tikv] table:t28064, index:iabc(a, b, c) range:[123 234 345,123 234 345], keep order:false, stats:pseudo", - " └─TableRowIDScan_6(Probe) 0.00 cop[tikv] table:t28064 keep order:false, stats:pseudo")) + rows.Check(testkit.Rows( + "IndexLookUp_7 0.00 root ", + "├─IndexRangeScan_5(Build) 0.00 cop[tikv] table:t28064, index:iabc(a, b, c) range:[123 234 345,123 234 345], keep order:false, stats:pseudo", + "└─TableRowIDScan_6(Probe) 0.00 cop[tikv] table:t28064 keep order:false, stats:pseudo")) tk.MustExec("execute stmt1 using @a, @b, @c;") rows = tk.MustQuery("select @@last_plan_from_cache") @@ -221,10 +219,10 @@ func TestIssue28064(t *testing.T) { tk.MustExec("execute stmt1 using @a, @b, @c;") rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) - rows.Check(testkit.Rows("Selection_8 0.00 root eq(test.t28064.a, 123), eq(test.t28064.b, 234), eq(test.t28064.c, 345)", - "└─IndexLookUp_7 0.00 root ", - " ├─IndexRangeScan_5(Build) 0.00 cop[tikv] table:t28064, index:iabc(a, b, c) range:[123 234 345,123 234 345], keep order:false, stats:pseudo", - " └─TableRowIDScan_6(Probe) 0.00 cop[tikv] table:t28064 keep order:false, stats:pseudo")) + rows.Check(testkit.Rows( + "IndexLookUp_7 0.00 root ", + "├─IndexRangeScan_5(Build) 0.00 cop[tikv] table:t28064, index:iabc(a, b, c) range:[123 234 345,123 234 345], keep order:false, stats:pseudo", + "└─TableRowIDScan_6(Probe) 0.00 cop[tikv] table:t28064 keep order:false, stats:pseudo")) } func TestPreparePlanCache4Blacklist(t *testing.T) { @@ -910,10 +908,9 @@ func TestIssue29101(t *testing.T) { tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // can use index-join `StreamAgg_9 1.00 root funcs:count(distinct test.stock.s_i_id)->Column#11`, `└─IndexJoin_14 0.03 root inner join, inner:IndexLookUp_13, outer key:test.order_line.ol_i_id, inner key:test.stock.s_i_id, equal cond:eq(test.order_line.ol_i_id, test.stock.s_i_id)`, - ` ├─Selection_30(Build) 0.03 root eq(test.order_line.ol_d_id, 1), eq(test.order_line.ol_w_id, 391), ge(test.order_line.ol_o_id, 3038), lt(test.order_line.ol_o_id, 3058)`, - ` │ └─IndexLookUp_29 0.03 root `, - ` │ ├─IndexRangeScan_27(Build) 0.03 cop[tikv] table:order_line, index:PRIMARY(ol_w_id, ol_d_id, ol_o_id, ol_number) range:[391 1 3038,391 1 3058), keep order:false, stats:pseudo`, - ` │ └─TableRowIDScan_28(Probe) 0.03 cop[tikv] table:order_line keep order:false, stats:pseudo`, + ` ├─IndexLookUp_28(Build) 0.03 root `, + ` │ ├─IndexRangeScan_26(Build) 0.03 cop[tikv] table:order_line, index:PRIMARY(ol_w_id, ol_d_id, ol_o_id, ol_number) range:[391 1 3038,391 1 3058), keep order:false, stats:pseudo`, + ` │ └─TableRowIDScan_27(Probe) 0.03 cop[tikv] table:order_line keep order:false, stats:pseudo`, ` └─IndexLookUp_13(Probe) 0.03 root `, ` ├─IndexRangeScan_10(Build) 0.03 cop[tikv] table:stock, index:PRIMARY(s_w_id, s_i_id) range: decided by [eq(test.stock.s_i_id, test.order_line.ol_i_id) eq(test.stock.s_w_id, 391)], keep order:false, stats:pseudo`, ` └─Selection_12(Probe) 0.03 cop[tikv] lt(test.stock.s_quantity, 18)`, diff --git a/executor/revoke.go b/executor/revoke.go index 337e387c5b28f..9063206ffd52a 100644 --- a/executor/revoke.go +++ b/executor/revoke.go @@ -180,6 +180,9 @@ func (e *RevokeExec) revokeOneUser(internalSession sessionctx.Context, user, hos } func (e *RevokeExec) revokePriv(internalSession sessionctx.Context, priv *ast.PrivElem, user, host string) error { + if priv.Priv == mysql.UsagePriv { + return nil + } switch e.Level.Level { case ast.GrantLevelGlobal: return e.revokeGlobalPriv(internalSession, priv, user, host) diff --git a/executor/revoke_test.go b/executor/revoke_test.go index 635fa18552df5..fcc53b5d291fe 100644 --- a/executor/revoke_test.go +++ b/executor/revoke_test.go @@ -271,3 +271,18 @@ func TestRevokeOnNonExistTable(t *testing.T) { tk.MustExec("DROP TABLE t1;") tk.MustExec("REVOKE ALTER ON d1.t1 FROM issue28533;") } + +// Check https://github.com/pingcap/tidb/issues/41773. +func TestIssue41773(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table if not exists xx (id int)") + tk.MustExec("CREATE USER 't1234'@'%' IDENTIFIED BY 'sNGNQo12fEHe0n3vU';") + tk.MustExec("GRANT USAGE ON * TO 't1234'@'%';") + tk.MustExec("GRANT USAGE ON test.* TO 't1234'@'%';") + tk.MustExec("GRANT USAGE ON test.xx TO 't1234'@'%';") + tk.MustExec("REVOKE USAGE ON * FROM 't1234'@'%';") + tk.MustExec("REVOKE USAGE ON test.* FROM 't1234'@'%';") + tk.MustExec("REVOKE USAGE ON test.xx FROM 't1234'@'%';") +} diff --git a/executor/slow_query_sql_test.go b/executor/slow_query_sql_test.go index cdf13f2a9ccf9..24644c8eebc2e 100644 --- a/executor/slow_query_sql_test.go +++ b/executor/slow_query_sql_test.go @@ -82,6 +82,46 @@ func TestSlowQuerySensitiveQuery(t *testing.T) { )) } +func TestSlowQueryNonPrepared(t *testing.T) { + originCfg := config.GetGlobalConfig() + newCfg := *originCfg + + f, err := os.CreateTemp("", "tidb-slow-*.log") + require.NoError(t, err) + require.NoError(t, f.Close()) + newCfg.Log.SlowQueryFile = f.Name() + config.StoreGlobalConfig(&newCfg) + defer func() { + config.StoreGlobalConfig(originCfg) + require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) + }() + require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + defer func() { + tk.MustExec("set tidb_slow_log_threshold=300;") + tk.MustExec("set tidb_redact_log=0;") + }() + + tk.MustExec(`use test`) + tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) + tk.MustExec(`create table t (a int)`) + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + tk.MustExec("set tidb_slow_log_threshold=0;") + + tk.MustExec(`select * from t where a<1`) + tk.MustExec(`select * from t where a<2`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=0`) + tk.MustExec(`select * from t where a<3`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + + tk.MustQuery(`select prepared, plan_from_cache, query from information_schema.slow_query where query like '%select * from t where a%' order by query`).Check(testkit.Rows( + `0 0 select * from t where a<1 [arguments: 1];`, + `0 1 select * from t where a<2 [arguments: 2];`, + `0 0 select * from t where a<3;`)) +} + func TestSlowQueryPrepared(t *testing.T) { originCfg := config.GetGlobalConfig() newCfg := *originCfg diff --git a/executor/testdata/prepare_suite_out.json b/executor/testdata/prepare_suite_out.json index 93adc89312160..387bff035be40 100644 --- a/executor/testdata/prepare_suite_out.json +++ b/executor/testdata/prepare_suite_out.json @@ -83,9 +83,8 @@ ], "Plan": [ "Projection_4 10.00 root test.t1.a", - "└─Selection_7 10.00 root eq(test.t1.b, 3)", - " └─IndexReader_6 10.00 root index:IndexRangeScan_5", - " └─IndexRangeScan_5 10.00 cop[tikv] table:t1, index:b(b, a) range:[3,3], keep order:false, stats:pseudo" + "└─IndexReader_6 10.00 root index:IndexRangeScan_5", + " └─IndexRangeScan_5 10.00 cop[tikv] table:t1, index:b(b, a) range:[3,3], keep order:false, stats:pseudo" ], "LastPlanUseCache": "0", "Result": [ @@ -102,9 +101,8 @@ ], "Plan": [ "Projection_4 10.00 root test.t1.a", - "└─Selection_7 10.00 root eq(test.t1.b, 2)", - " └─IndexReader_6 10.00 root index:IndexRangeScan_5", - " └─IndexRangeScan_5 10.00 cop[tikv] table:t1, index:b(b, a) range:[2,2], keep order:false, stats:pseudo" + "└─IndexReader_6 10.00 root index:IndexRangeScan_5", + " └─IndexRangeScan_5 10.00 cop[tikv] table:t1, index:b(b, a) range:[2,2], keep order:false, stats:pseudo" ], "LastPlanUseCache": "1", "Result": [ @@ -121,9 +119,8 @@ ], "Plan": [ "Projection_4 10.00 root test.t1.a", - "└─Selection_7 10.00 root eq(test.t1.b, -200)", - " └─IndexReader_6 10.00 root index:IndexRangeScan_5", - " └─IndexRangeScan_5 10.00 cop[tikv] table:t1, index:b(b, a) range:[-200,-200], keep order:false, stats:pseudo" + "└─IndexReader_6 10.00 root index:IndexRangeScan_5", + " └─IndexRangeScan_5 10.00 cop[tikv] table:t1, index:b(b, a) range:[-200,-200], keep order:false, stats:pseudo" ], "LastPlanUseCache": "1", "Result": null @@ -159,16 +156,14 @@ ], "Plan": [ "HashJoin_38 6387.21 root inner join, equal:[eq(test.t1.b, test.t2.b) eq(test.t1.a, test.t2.a)]", - "├─Selection_69(Build) 79.92 root eq(test.t2.b, 1), not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ └─IndexLookUp_68 99.80 root ", - "│ ├─Selection_67(Build) 99.80 cop[tikv] not(isnull(test.t2.b))", - "│ │ └─IndexRangeScan_65 99.90 cop[tikv] table:t2, index:b(b, a) range:[1 -inf,1 +inf], keep order:false, stats:pseudo", - "│ └─TableRowIDScan_66(Probe) 99.80 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─Selection_60(Probe) 79.92 root eq(test.t1.b, 1), not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " └─IndexLookUp_59 99.80 root ", - " ├─Selection_58(Build) 99.80 cop[tikv] not(isnull(test.t1.b))", - " │ └─IndexRangeScan_56 99.90 cop[tikv] table:t1, index:b(b, a) range:[1 -inf,1 +inf], keep order:false, stats:pseudo", - " └─TableRowIDScan_57(Probe) 99.80 cop[tikv] table:t1 keep order:false, stats:pseudo" + "├─IndexLookUp_63(Build) 99.80 root ", + "│ ├─Selection_62(Build) 99.80 cop[tikv] not(isnull(test.t2.b))", + "│ │ └─IndexRangeScan_60 99.90 cop[tikv] table:t2, index:b(b, a) range:[1 -inf,1 +inf], keep order:false, stats:pseudo", + "│ └─TableRowIDScan_61(Probe) 99.80 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp_56(Probe) 99.80 root ", + " ├─Selection_55(Build) 99.80 cop[tikv] not(isnull(test.t1.b))", + " │ └─IndexRangeScan_53 99.90 cop[tikv] table:t1, index:b(b, a) range:[1 -inf,1 +inf], keep order:false, stats:pseudo", + " └─TableRowIDScan_54(Probe) 99.80 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "LastPlanUseCache": "0", "Result": null @@ -183,16 +178,14 @@ ], "Plan": [ "HashJoin_38 6387.21 root inner join, equal:[eq(test.t1.b, test.t2.b) eq(test.t1.a, test.t2.a)]", - "├─Selection_69(Build) 79.92 root eq(test.t2.b, 2), not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ └─IndexLookUp_68 99.80 root ", - "│ ├─Selection_67(Build) 99.80 cop[tikv] not(isnull(test.t2.b))", - "│ │ └─IndexRangeScan_65 99.90 cop[tikv] table:t2, index:b(b, a) range:[2 -inf,2 +inf], keep order:false, stats:pseudo", - "│ └─TableRowIDScan_66(Probe) 99.80 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─Selection_60(Probe) 79.92 root eq(test.t1.b, 2), not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " └─IndexLookUp_59 99.80 root ", - " ├─Selection_58(Build) 99.80 cop[tikv] not(isnull(test.t1.b))", - " │ └─IndexRangeScan_56 99.90 cop[tikv] table:t1, index:b(b, a) range:[2 -inf,2 +inf], keep order:false, stats:pseudo", - " └─TableRowIDScan_57(Probe) 99.80 cop[tikv] table:t1 keep order:false, stats:pseudo" + "├─IndexLookUp_63(Build) 99.80 root ", + "│ ├─Selection_62(Build) 99.80 cop[tikv] not(isnull(test.t2.b))", + "│ │ └─IndexRangeScan_60 99.90 cop[tikv] table:t2, index:b(b, a) range:[2 -inf,2 +inf], keep order:false, stats:pseudo", + "│ └─TableRowIDScan_61(Probe) 99.80 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─IndexLookUp_56(Probe) 99.80 root ", + " ├─Selection_55(Build) 99.80 cop[tikv] not(isnull(test.t1.b))", + " │ └─IndexRangeScan_53 99.90 cop[tikv] table:t1, index:b(b, a) range:[2 -inf,2 +inf], keep order:false, stats:pseudo", + " └─TableRowIDScan_54(Probe) 99.80 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "LastPlanUseCache": "1", "Result": [ @@ -1086,10 +1079,9 @@ "6" ], "Plan": [ - "Selection_8 8000.00 root gt(plus(test.t.a, 0), 1)", - "└─IndexReader_7 8000.00 root index:Selection_6", - " └─Selection_6 8000.00 cop[tikv] gt(plus(test.t.a, 0), 1)", - " └─IndexFullScan_5 10000.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo" + "IndexReader_7 8000.00 root index:Selection_6", + "└─Selection_6 8000.00 cop[tikv] gt(plus(test.t.a, 0), 1)", + " └─IndexFullScan_5 10000.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo" ], "FromCache": "0" }, @@ -1098,10 +1090,9 @@ "6" ], "Plan": [ - "Selection_8 8000.00 root gt(plus(test.t.a, 0), 5)", - "└─IndexReader_7 8000.00 root index:Selection_6", - " └─Selection_6 8000.00 cop[tikv] gt(plus(test.t.a, 0), 5)", - " └─IndexFullScan_5 10000.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo" + "IndexReader_7 8000.00 root index:Selection_6", + "└─Selection_6 8000.00 cop[tikv] gt(plus(test.t.a, 0), 5)", + " └─IndexFullScan_5 10000.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo" ], "FromCache": "1" }, @@ -1119,10 +1110,9 @@ "6 6 6" ], "Plan": [ - "Selection_8 3333.33 root gt(test.t.b, 1)", - "└─TableReader_7 3333.33 root data:Selection_6", - " └─Selection_6 3333.33 cop[tikv] gt(test.t.b, 1)", - " └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + "TableReader_7 3333.33 root data:Selection_6", + "└─Selection_6 3333.33 cop[tikv] gt(test.t.b, 1)", + " └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "FromCache": "0" }, @@ -1131,10 +1121,9 @@ "6 6 6" ], "Plan": [ - "Selection_8 3333.33 root gt(test.t.b, 5)", - "└─TableReader_7 3333.33 root data:Selection_6", - " └─Selection_6 3333.33 cop[tikv] gt(test.t.b, 5)", - " └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + "TableReader_7 3333.33 root data:Selection_6", + "└─Selection_6 3333.33 cop[tikv] gt(test.t.b, 5)", + " └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "FromCache": "1" }, @@ -1152,12 +1141,11 @@ "6 6 6" ], "Plan": [ - "Selection_10 2666.67 root gt(plus(test.t.a, 0), 1), gt(test.t.b, 1)", - "└─IndexLookUp_9 2666.67 root ", - " ├─Selection_7(Build) 8000.00 cop[tikv] gt(plus(test.t.a, 0), 1)", - " │ └─IndexFullScan_5 10000.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", - " └─Selection_8(Probe) 2666.67 cop[tikv] gt(test.t.b, 1)", - " └─TableRowIDScan_6 8000.00 cop[tikv] table:t keep order:false, stats:pseudo" + "IndexLookUp_9 2666.67 root ", + "├─Selection_7(Build) 8000.00 cop[tikv] gt(plus(test.t.a, 0), 1)", + "│ └─IndexFullScan_5 10000.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + "└─Selection_8(Probe) 2666.67 cop[tikv] gt(test.t.b, 1)", + " └─TableRowIDScan_6 8000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "FromCache": "0" }, @@ -1166,12 +1154,11 @@ "6 6 6" ], "Plan": [ - "Selection_10 2666.67 root gt(plus(test.t.a, 0), 5), gt(test.t.b, 5)", - "└─IndexLookUp_9 2666.67 root ", - " ├─Selection_7(Build) 8000.00 cop[tikv] gt(plus(test.t.a, 0), 5)", - " │ └─IndexFullScan_5 10000.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", - " └─Selection_8(Probe) 2666.67 cop[tikv] gt(test.t.b, 5)", - " └─TableRowIDScan_6 8000.00 cop[tikv] table:t keep order:false, stats:pseudo" + "IndexLookUp_9 2666.67 root ", + "├─Selection_7(Build) 8000.00 cop[tikv] gt(plus(test.t.a, 0), 5)", + "│ └─IndexFullScan_5 10000.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + "└─Selection_8(Probe) 2666.67 cop[tikv] gt(test.t.b, 5)", + " └─TableRowIDScan_6 8000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "FromCache": "1" }, diff --git a/executor/utils.go b/executor/utils.go index 47fe32a93aa68..4997eb9d277a8 100644 --- a/executor/utils.go +++ b/executor/utils.go @@ -16,6 +16,8 @@ package executor import ( "strings" + + "github.com/pingcap/errors" ) // SetFromString constructs a slice of strings from a comma separated string. @@ -92,3 +94,20 @@ func (b *batchRetrieverHelper) nextBatch(retrieveRange func(start, end int) erro } return nil } + +// TODO: add GetMsg() to errors package to replace this function. +// see TestGetMsgFromBRError for more details. +func getMsgFromBRError(err error) string { + if err == nil { + return "" + } + if berr, ok := err.(*errors.Error); ok { + return berr.GetMsg() + } + raw := err.Error() + berrMsg := errors.Cause(err).Error() + if len(raw) <= len(berrMsg)+len(": ") { + return raw + } + return raw[:len(raw)-len(berrMsg)-len(": ")] +} diff --git a/executor/utils_test.go b/executor/utils_test.go index 3c8a32de25cc5..e0795228141a5 100644 --- a/executor/utils_test.go +++ b/executor/utils_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/pingcap/errors" + berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/stretchr/testify/require" ) @@ -93,3 +94,12 @@ func TestBatchRetrieverHelper(t *testing.T) { require.Equal(t, rangeStarts, []int{0}) require.Equal(t, rangeEnds, []int{10}) } + +func TestGetMsgFromBRError(t *testing.T) { + var berr error = berrors.ErrStorageInvalidConfig + require.Equal(t, "[BR:ExternalStorage:ErrStorageInvalidConfig]invalid external storage config", berr.Error()) + require.Equal(t, "invalid external storage config", getMsgFromBRError(berr)) + berr = errors.Annotatef(berr, "some message about error reason") + require.Equal(t, "some message about error reason: [BR:ExternalStorage:ErrStorageInvalidConfig]invalid external storage config", berr.Error()) + require.Equal(t, "some message about error reason", getMsgFromBRError(berr)) +} diff --git a/executor/writetest/write_test.go b/executor/writetest/write_test.go index 021e678708817..fdd8991a10f2e 100644 --- a/executor/writetest/write_test.go +++ b/executor/writetest/write_test.go @@ -1874,7 +1874,7 @@ type testCase struct { func checkCases( tests []testCase, - ld *executor.LoadDataInfo, + ld *executor.LoadDataWorker, t *testing.T, tk *testkit.TestKit, ctx sessionctx.Context, @@ -1903,7 +1903,7 @@ func checkCases( require.NoError(t, err1) err1 = ld.CheckAndInsertOneBatch(context.Background(), ld.GetRows(), ld.GetCurBatchCnt()) require.NoError(t, err1) - ld.SetMaxRowsInBatch(20000) + ld.ResetBatch() ld.SetMessage() require.Equal(t, tt.expectedMsg, tk.Session().LastMessage()) ctx.StmtCommit(context.Background()) @@ -1924,7 +1924,7 @@ func TestLoadDataMissingColumn(t *testing.T) { tk.MustExec(createSQL) tk.MustExec("load data local infile '/tmp/nonexistence.csv' ignore into table load_data_missing") ctx := tk.Session().(sessionctx.Context) - ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataWorker) require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) require.NotNil(t, ld) @@ -1974,7 +1974,7 @@ func TestIssue18681(t *testing.T) { tk.MustExec(createSQL) tk.MustExec("load data local infile '/tmp/nonexistence.csv' ignore into table load_data_test") ctx := tk.Session().(sessionctx.Context) - ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataWorker) require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) require.NotNil(t, ld) @@ -1983,7 +1983,7 @@ func TestIssue18681(t *testing.T) { selectSQL := "select bin(a), bin(b), bin(c), bin(d) from load_data_test;" ctx.GetSessionVars().StmtCtx.DupKeyAsWarning = true ctx.GetSessionVars().StmtCtx.BadNullAsWarning = true - ld.SetMaxRowsInBatch(20000) + ld.ResetBatch() sc := ctx.GetSessionVars().StmtCtx originIgnoreTruncate := sc.IgnoreTruncate @@ -2026,7 +2026,7 @@ func TestIssue34358(t *testing.T) { tk.MustExec("create table load_data_test (a varchar(10), b varchar(10))") tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table load_data_test ( @v1, @v2 ) set a = @v1, b = @v2") - ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataWorker) require.True(t, ok) require.NotNil(t, ld) checkCases([]testCase{ @@ -2048,7 +2048,7 @@ func TestLoadData(t *testing.T) { require.Error(t, err) tk.MustExec("load data local infile '/tmp/nonexistence.csv' ignore into table load_data_test") ctx := tk.Session().(sessionctx.Context) - ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataWorker) require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) require.NotNil(t, ld) @@ -2070,7 +2070,7 @@ func TestLoadData(t *testing.T) { require.NoError(t, err) err = ld.CheckAndInsertOneBatch(context.Background(), ld.GetRows(), ld.GetCurBatchCnt()) require.NoError(t, err) - ld.SetMaxRowsInBatch(20000) + ld.ResetBatch() r := tk.MustQuery(selectSQL) r.Check(nil) @@ -2229,7 +2229,7 @@ func TestLoadDataEscape(t *testing.T) { tk.MustExec("CREATE TABLE load_data_test (id INT NOT NULL PRIMARY KEY, value TEXT NOT NULL) CHARACTER SET utf8") tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table load_data_test") ctx := tk.Session().(sessionctx.Context) - ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataWorker) require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) require.NotNil(t, ld) @@ -2260,7 +2260,7 @@ func TestLoadDataSpecifiedColumns(t *testing.T) { tk.MustExec(`create table load_data_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 varchar(255) default "def", c3 int default 0);`) tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table load_data_test (c1, c2)") ctx := tk.Session().(sessionctx.Context) - ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataWorker) require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) require.NotNil(t, ld) @@ -2286,7 +2286,7 @@ func TestLoadDataIgnoreLines(t *testing.T) { tk.MustExec("CREATE TABLE load_data_test (id INT NOT NULL PRIMARY KEY, value TEXT NOT NULL) CHARACTER SET utf8") tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table load_data_test ignore 1 lines") ctx := tk.Session().(sessionctx.Context) - ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataWorker) require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) require.NotNil(t, ld) @@ -2311,7 +2311,7 @@ func TestLoadDataNULL(t *testing.T) { tk.MustExec(`load data local infile '/tmp/nonexistence.csv' into table load_data_test FIELDS TERMINATED BY ',' ENCLOSED BY '"' LINES TERMINATED BY '\n';`) ctx := tk.Session().(sessionctx.Context) - ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataWorker) require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) require.NotNil(t, ld) @@ -2338,7 +2338,7 @@ func TestLoadDataReplace(t *testing.T) { tk.MustExec("INSERT INTO load_data_replace VALUES(1,'val 1'),(2,'val 2')") tk.MustExec("LOAD DATA LOCAL INFILE '/tmp/nonexistence.csv' REPLACE INTO TABLE load_data_replace") ctx := tk.Session().(sessionctx.Context) - ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataWorker) require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) require.NotNil(t, ld) @@ -2359,7 +2359,7 @@ func TestLoadDataOverflowBigintUnsigned(t *testing.T) { tk.MustExec("CREATE TABLE load_data_test (a bigint unsigned);") tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table load_data_test") ctx := tk.Session().(sessionctx.Context) - ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataWorker) require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) require.NotNil(t, ld) @@ -2380,7 +2380,7 @@ func TestLoadDataWithUppercaseUserVars(t *testing.T) { tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table load_data_test (@V1)" + " set a = @V1, b = @V1*100") ctx := tk.Session().(sessionctx.Context) - ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataWorker) require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) require.NotNil(t, ld) @@ -2402,7 +2402,7 @@ func TestLoadDataIntoPartitionedTable(t *testing.T) { "partition p2 values less than (11))") tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table range_t fields terminated by ','") ctx := tk.Session().(sessionctx.Context) - ld := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + ld := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataWorker) require.Nil(t, sessiontxn.NewTxn(context.Background(), ctx)) parser, err := mydump.NewCSVParser( @@ -2419,7 +2419,7 @@ func TestLoadDataIntoPartitionedTable(t *testing.T) { require.NoError(t, err) err = ld.CheckAndInsertOneBatch(context.Background(), ld.GetRows(), ld.GetCurBatchCnt()) require.NoError(t, err) - ld.SetMaxRowsInBatch(20000) + ld.ResetBatch() ld.SetMessage() ctx.StmtCommit(context.Background()) txn, err := ctx.Txn(true) diff --git a/go.mod b/go.mod index b15c878e18602..7258b1622ee25 100644 --- a/go.mod +++ b/go.mod @@ -26,7 +26,7 @@ require ( github.com/cockroachdb/pebble v0.0.0-20210719141320-8c3bd06debb5 github.com/coocood/freecache v1.2.1 github.com/coreos/go-semver v0.3.0 - github.com/daixiang0/gci v0.9.0 + github.com/daixiang0/gci v0.9.1 github.com/danjacques/gofslock v0.0.0-20191023191349-0a45f885bc37 github.com/dgraph-io/ristretto v0.1.1 github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 @@ -41,7 +41,7 @@ require ( github.com/golang/protobuf v1.5.2 github.com/golang/snappy v0.0.4 github.com/golangci/gofmt v0.0.0-20220901101216-f2edd75033f2 - github.com/golangci/golangci-lint v1.51.1 + github.com/golangci/golangci-lint v1.51.2 github.com/golangci/gosec v0.0.0-20180901114220-8afd9cbb6cfb github.com/golangci/misspell v0.4.0 github.com/golangci/prealloc v0.0.0-20180630174525-215b22d4de21 @@ -92,7 +92,7 @@ require ( github.com/stretchr/testify v1.8.1 github.com/tdakkota/asciicheck v0.1.1 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.6-0.20230222073543-a27994e5aae2 + github.com/tikv/client-go/v2 v2.0.6-0.20230228054731-11d34cbd8a7f github.com/tikv/pd/client v0.0.0-20230209034200-6d23a31c24be github.com/timakin/bodyclose v0.0.0-20221125081123-e39cf3fc478e github.com/twmb/murmur3 v1.1.6 @@ -262,7 +262,7 @@ require ( go.opentelemetry.io/otel/trace v0.20.0 // indirect go.opentelemetry.io/proto/otlp v0.7.0 // indirect golang.org/x/crypto v0.6.0 // indirect - golang.org/x/exp/typeparams v0.0.0-20221208152030-732eee02a75a // indirect + golang.org/x/exp/typeparams v0.0.0-20230203172020-98cc5a0785f9 // indirect golang.org/x/mod v0.8.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect google.golang.org/appengine v1.6.7 // indirect diff --git a/go.sum b/go.sum index 1ffd978e8848d..4524ca36ccdff 100644 --- a/go.sum +++ b/go.sum @@ -216,8 +216,8 @@ github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ3 github.com/creack/pty v1.1.11/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= -github.com/daixiang0/gci v0.9.0 h1:t8XZ0vK6l0pwPoOmoGyqW2NwQlvbpAQNVvu/GRBgykM= -github.com/daixiang0/gci v0.9.0/go.mod h1:EpVfrztufwVgQRXjnX4zuNinEpLj5OmMjtu/+MB0V0c= +github.com/daixiang0/gci v0.9.1 h1:jBrwBmBZTDsGsXiaCTLIe9diotp1X4X64zodFrh7l+c= +github.com/daixiang0/gci v0.9.1/go.mod h1:EpVfrztufwVgQRXjnX4zuNinEpLj5OmMjtu/+MB0V0c= github.com/danjacques/gofslock v0.0.0-20191023191349-0a45f885bc37 h1:X6mKGhCFOxrKeeHAjv/3UvT6e5RRxW6wRdlqlV6/H4w= github.com/danjacques/gofslock v0.0.0-20191023191349-0a45f885bc37/go.mod h1:DC3JtzuG7kxMvJ6dZmf2ymjNyoXwgtklr7FN+Um2B0U= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -387,8 +387,8 @@ github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golangci/gofmt v0.0.0-20220901101216-f2edd75033f2 h1:amWTbTGqOZ71ruzrdA+Nx5WA3tV1N0goTspwmKCQvBY= github.com/golangci/gofmt v0.0.0-20220901101216-f2edd75033f2/go.mod h1:9wOXstvyDRshQ9LggQuzBCGysxs3b6Uo/1MvYCR2NMs= -github.com/golangci/golangci-lint v1.51.1 h1:N5HD/x0ZrhJYsgKWyz7yJxxQ8JKR0Acc+FOP7QtGSAA= -github.com/golangci/golangci-lint v1.51.1/go.mod h1:hnyNNO3fJ2Rjwo6HM+VXvcmLkKDOuBAnR9gVlS1mW1E= +github.com/golangci/golangci-lint v1.51.2 h1:yIcsT1X9ZYHdSpeWXRT1ORC/FPGSqDHbHsu9uk4FK7M= +github.com/golangci/golangci-lint v1.51.2/go.mod h1:KH9Q7/3glwpYSknxUgUyLlAv46A8fsSKo1hH2wDvkr8= github.com/golangci/gosec v0.0.0-20180901114220-8afd9cbb6cfb h1:Bi7BYmZVg4C+mKGi8LeohcP2GGUl2XJD4xCkJoZSaYc= github.com/golangci/gosec v0.0.0-20180901114220-8afd9cbb6cfb/go.mod h1:ON/c2UR0VAAv6ZEAFKhjCLplESSmRFfZcDLASbI1GWo= github.com/golangci/misspell v0.4.0 h1:KtVB/hTK4bbL/S6bs64rYyk8adjmh1BygbBiaAiX+a0= @@ -937,8 +937,8 @@ github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJf github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW93SG+q0F8KI+yFrcIDT4c/RNoc4= github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a/go.mod h1:h4xBhSNtOeEosLJ4P7JyKXX7Cabg7AVkWCK5gV2vOrM= -github.com/tikv/client-go/v2 v2.0.6-0.20230222073543-a27994e5aae2 h1:HgU1sa4pzjduuMAMysbfAt8ebx5p2SZyPFu/DC0Ir0c= -github.com/tikv/client-go/v2 v2.0.6-0.20230222073543-a27994e5aae2/go.mod h1:Kw6+qOCoLSymJLtOPTym6wkH97Ej0Yna4pDv0ak3hMc= +github.com/tikv/client-go/v2 v2.0.6-0.20230228054731-11d34cbd8a7f h1:hDbgZE2B2ZNZ4ID8E57Z/glM/xXAy11L00S554NRSx8= +github.com/tikv/client-go/v2 v2.0.6-0.20230228054731-11d34cbd8a7f/go.mod h1:E4pTR2qYbxgu9PKrqhOrxM2+yXEcKObssBltiZ6Gc5Y= github.com/tikv/pd/client v0.0.0-20230209034200-6d23a31c24be h1:hauBQBHSyrUxAI0zvkTiBKd472c+Iy+aY0Jd+b9VOJ8= github.com/tikv/pd/client v0.0.0-20230209034200-6d23a31c24be/go.mod h1:ryhYHDwupsZHeOOF/N7So+1hbtAnuw0K2A+pKOElSVs= github.com/timakin/bodyclose v0.0.0-20221125081123-e39cf3fc478e h1:MV6KaVu/hzByHP0UvJ4HcMGE/8a6A4Rggc/0wx2AvJo= @@ -1110,8 +1110,8 @@ golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMk golang.org/x/exp v0.0.0-20200513190911-00229845015e/go.mod h1:4M0jN8W1tt0AVLNr8HDosyJCDCDuyL9N9+3m7wDWgKw= golang.org/x/exp v0.0.0-20221023144134-a1e5550cf13e h1:SkwG94eNiiYJhbeDE018Grw09HIN/KB9NlRmZsrzfWs= golang.org/x/exp v0.0.0-20221023144134-a1e5550cf13e/go.mod h1:cyybsKvd6eL0RnXn6p/Grxp8F5bW7iYuBgsNCOHpMYE= -golang.org/x/exp/typeparams v0.0.0-20221208152030-732eee02a75a h1:Jw5wfR+h9mnIYH+OtGT2im5wV1YGGDora5vTv/aa5bE= -golang.org/x/exp/typeparams v0.0.0-20221208152030-732eee02a75a/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= +golang.org/x/exp/typeparams v0.0.0-20230203172020-98cc5a0785f9 h1:6WHiuFL9FNjg8RljAaT7FNUuKDbvMqS1i5cr2OE2sLQ= +golang.org/x/exp/typeparams v0.0.0-20230203172020-98cc5a0785f9/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= diff --git a/metrics/grafana/tidb_resource_control.json b/metrics/grafana/tidb_resource_control.json index a34e2c92e95ab..0d64321b32b3c 100644 --- a/metrics/grafana/tidb_resource_control.json +++ b/metrics/grafana/tidb_resource_control.json @@ -116,7 +116,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name) + sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name) + sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -126,7 +126,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) + sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) + sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "hide": false, "interval": "", "intervalFactor": 2, @@ -232,7 +232,7 @@ "targets": [ { "exemplar": true, - "expr": "(sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name) + sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "(sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name) + sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -242,7 +242,7 @@ }, { "exemplar": true, - "expr": "(sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) + sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "(sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) + sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "hide": false, "interval": "", @@ -350,7 +350,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -360,7 +360,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "hide": false, "instant": false, "interval": "", @@ -467,7 +467,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -477,7 +477,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(resource_manager_resource_unit_read_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "hide": false, "interval": "", @@ -585,7 +585,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -595,7 +595,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "hide": false, "interval": "", @@ -703,7 +703,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -713,7 +713,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(resource_manager_resource_unit_write_request_unit_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "hide": false, "interval": "", @@ -832,7 +832,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name, type)", + "expr": "sum(rate(resource_manager_resource_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name, type)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -842,7 +842,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "sum(rate(resource_manager_resource_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "hide": false, "interval": "", @@ -950,7 +950,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"read\"}[30s])) by (name) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "sum(rate(resource_manager_resource_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"read\"}[1m])) by (name) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -960,7 +960,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"write\"}[30s])) by (name) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "sum(rate(resource_manager_resource_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"write\"}[1m])) by (name) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "hide": false, "interval": "", @@ -971,7 +971,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"read\"}[30s])) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(resource_manager_resource_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"read\"}[1m])) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -981,7 +981,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"write\"}[30s])) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(resource_manager_resource_request_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"write\"}[1m])) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "hide": false, "interval": "", @@ -1089,7 +1089,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_read_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "sum(rate(resource_manager_resource_read_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -1099,7 +1099,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_read_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(resource_manager_resource_read_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "hide": false, "interval": "", @@ -1208,7 +1208,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_read_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "sum(rate(resource_manager_resource_read_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -1218,7 +1218,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_read_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(resource_manager_resource_read_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "hide": false, "interval": "", @@ -1327,7 +1327,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_write_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "sum(rate(resource_manager_resource_write_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -1337,7 +1337,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_write_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(resource_manager_resource_write_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "hide": false, "interval": "", @@ -1445,7 +1445,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_write_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "sum(rate(resource_manager_resource_write_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -1455,7 +1455,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_write_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(resource_manager_resource_write_byte_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "hide": false, "interval": "", @@ -1563,7 +1563,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_kv_cpu_time_ms_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "sum(rate(resource_manager_resource_kv_cpu_time_ms_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -1573,7 +1573,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_kv_cpu_time_ms_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(resource_manager_resource_kv_cpu_time_ms_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "hide": false, "interval": "", @@ -1682,7 +1682,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_kv_cpu_time_ms_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", + "expr": "sum(rate(resource_manager_resource_sql_cpu_time_ms_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (name)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -1692,126 +1692,7 @@ }, { "exemplar": true, - "expr": "sum(rate(resource_manager_resource_kv_cpu_time_ms_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "total", - "refId": "B", - "step": 40 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "KV CPU Time Per Query", - "tooltip": { - "msResolution": true, - "shared": true, - "sort": 0, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "$$hashKey": "object:869", - "decimals": null, - "format": "ms", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "$$hashKey": "object:870", - "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": "", - "editable": true, - "error": false, - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 1, - "fillGradient": 0, - "grid": {}, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 55 - }, - "hiddenSeries": false, - "id": 27, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": true, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 2, - "links": [], - "nullPointMode": "null as zero", - "options": { - "alertThreshold": true - }, - "percentage": false, - "pluginVersion": "7.5.10", - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "exemplar": true, - "expr": "sum(rate(resource_manager_resource_sql_cpu_time_ms_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", - "format": "time_series", - "interval": "", - "intervalFactor": 2, - "legendFormat": "{{name}}", - "refId": "A", - "step": 40 - }, - { - "exemplar": true, - "expr": "sum(rate(resource_manager_resource_sql_cpu_time_ms_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(resource_manager_resource_sql_cpu_time_ms_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "hide": false, "interval": "", @@ -1843,125 +1724,7 @@ "yaxes": [ { "$$hashKey": "object:869", - "format": "bits", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "$$hashKey": "object:870", - "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": "", - "editable": true, - "error": false, - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 1, - "fillGradient": 0, - "grid": {}, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 55 - }, - "hiddenSeries": false, - "id": 28, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": true, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 2, - "links": [], - "nullPointMode": "null as zero", - "options": { - "alertThreshold": true - }, - "percentage": false, - "pluginVersion": "7.5.10", - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "exemplar": true, - "expr": "sum(rate(resource_manager_resource_sql_cpu_time_ms_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (name)", - "format": "time_series", - "interval": "", - "intervalFactor": 2, - "legendFormat": "{{name}}", - "refId": "A", - "step": 40 - }, - { - "exemplar": true, - "expr": "sum(rate(resource_manager_resource_sql_cpu_time_ms_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) / sum(rate(tidb_session_resource_group_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "total", - "refId": "B", - "step": 40 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "SQL CPU Time Per Query", - "tooltip": { - "msResolution": true, - "shared": true, - "sort": 0, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "$$hashKey": "object:869", - "format": "bits", + "format": "ms", "label": null, "logBase": 1, "max": null, diff --git a/metrics/log_backup.go b/metrics/log_backup.go index 767fe2e251d8a..d9fc17d42d74c 100644 --- a/metrics/log_backup.go +++ b/metrics/log_backup.go @@ -60,10 +60,11 @@ var ( Name: "region_request_failure", Help: "The failure reasons of requesting region checkpoints.", }, []string{"reason"}) - RegionCheckpointSubscriptionEvent = prometheus.NewCounterVec(prometheus.CounterOpts{ + RegionCheckpointSubscriptionEvent = prometheus.NewHistogramVec(prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "log_backup", Name: "region_checkpoint_event", - Help: "The region flush event count.", + Help: "The region flush event size.", + Buckets: prometheus.ExponentialBuckets(8, 2.0, 12), }, []string{"store"}) ) diff --git a/parser/go.mod b/parser/go.mod index 1f49f53d36814..bd63a46fe8f0c 100644 --- a/parser/go.mod +++ b/parser/go.mod @@ -4,28 +4,27 @@ require ( github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 github.com/cznic/sortutil v0.0.0-20181122101858-f5f958428db8 github.com/cznic/strutil v0.0.0-20171016134553-529a34b1c186 - github.com/go-sql-driver/mysql v1.6.0 + github.com/go-sql-driver/mysql v1.7.0 github.com/pingcap/errors v0.11.5-0.20210425183316-da1aaba5fb63 github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7 - github.com/stretchr/testify v1.7.0 - go.uber.org/goleak v1.1.10 - go.uber.org/zap v1.18.1 + github.com/stretchr/testify v1.8.0 + go.uber.org/goleak v1.1.11 + go.uber.org/zap v1.24.0 golang.org/x/exp v0.0.0-20220428152302-39d4317da171 - golang.org/x/text v0.3.7 + golang.org/x/text v0.7.0 modernc.org/parser v1.0.2 modernc.org/y v1.0.1 ) require ( + github.com/benbjohnson/clock v1.1.0 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect go.uber.org/atomic v1.7.0 // indirect go.uber.org/multierr v1.6.0 // indirect - golang.org/x/lint v0.0.0-20190930215403-16217165b5de // indirect - golang.org/x/tools v0.1.8-0.20211029000441-d6a9af8af023 // indirect gopkg.in/natefinch/lumberjack.v2 v2.0.0 // indirect - gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b // indirect + gopkg.in/yaml.v3 v3.0.1 // indirect modernc.org/golex v1.0.1 // indirect modernc.org/mathutil v1.4.1 // indirect modernc.org/sortutil v1.0.0 // indirect diff --git a/parser/go.sum b/parser/go.sum index 267fe82580882..470e2c8276361 100644 --- a/parser/go.sum +++ b/parser/go.sum @@ -11,8 +11,8 @@ github.com/cznic/strutil v0.0.0-20171016134553-529a34b1c186/go.mod h1:AHHPPPXTw0 github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/go-sql-driver/mysql v1.6.0 h1:BCTh4TKNUYmOmMUcQ3IipzF5prigylS7XXjEkfCHuOE= -github.com/go-sql-driver/mysql v1.6.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= +github.com/go-sql-driver/mysql v1.7.0 h1:ueSltNNllEqE3qcWBTD0iQd3IpL/6U+mJxLkazJ7YPc= +github.com/go-sql-driver/mysql v1.7.0/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9S1MCJN5yJMI= github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= @@ -30,53 +30,69 @@ github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZN github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 h1:OdAsTTz6OkFY5QxjkYwrChwuRruF69c169dPK26NUlk= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.8.0 h1:pSgiaMZlXftHpm5L7V1+rVB+AZJydKsMxsQBIJw4PKk= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.10 h1:z+mqJhf6ss6BSfSM671tgKyZBFPTTJM+HLxnhPC3wu0= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= +go.uber.org/goleak v1.1.11 h1:wy28qYRKZgnJTxGxvye5/wgWr1EKjmUDGYox5mGlRlI= +go.uber.org/goleak v1.1.11/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.6.0 h1:y6IPFStTAIT5Ytl7/XYmHvzXQ7S3g/IeZW9hyZ5thw4= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.18.1 h1:CSUJ2mjFszzEWt4CdKISEuChVIXGBn3lAPwkRGyVrc4= -go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= +go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= +go.uber.org/zap v1.24.0/go.mod h1:2kMP+WWQ8aoFoedH3T2sq6iJ2yDWpHbP0f6MQbS9Gkg= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/exp v0.0.0-20181106170214-d68db9428509/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20220428152302-39d4317da171 h1:TfdoLivD44QwvssI9Sv1xwa5DcL5XQr4au4sZ2F2NV4= golang.org/x/exp v0.0.0-20220428152302-39d4317da171/go.mod h1:lgLbSvA5ygNOMpwM/9anMpWVlVJ7Z+cHWq/eFuinpGE= golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20211019181941-9d821ace8654 h1:id054HUawV2/6IGm2IV8KZQjqtwAOo2CYlOToYqa0d0= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= -golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.7.0 h1:4BRB4x83lYWy72KwLD/qYDuTu7q9PjSagHvijDw7cLo= +golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.1.8-0.20211029000441-d6a9af8af023 h1:0c3L82FDQ5rt1bjTBlchS8t6RQ6299/+5bWMnRLh+uI= -golang.org/x/tools v0.1.8-0.20211029000441-d6a9af8af023/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.12 h1:VveCTK38A2rkS8ZqFY25HIDFscX5X9OoEhJd3quQmXU= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= +gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8 h1:obN1ZagJSUGI0Ek/LBmuj4SNLPfIny3KsKFopxRdj10= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= modernc.org/fileutil v1.0.0/go.mod h1:JHsWpkrk/CnVV1H/eGlFf85BEpfkrp56ro8nojIq9Q8= modernc.org/golex v1.0.1 h1:EYKY1a3wStt0RzHaH8mdSRNg78Ub0OHxYfCRWw35YtM= modernc.org/golex v1.0.1/go.mod h1:QCA53QtsT1NdGkaZZkF5ezFwk4IXh4BGNafAARTC254= diff --git a/parser/model/BUILD.bazel b/parser/model/BUILD.bazel index 5387516deaf94..fa03be5b15c01 100644 --- a/parser/model/BUILD.bazel +++ b/parser/model/BUILD.bazel @@ -6,6 +6,7 @@ go_library( "ddl.go", "flags.go", "model.go", + "reorg.go", ], importpath = "github.com/pingcap/tidb/parser/model", visibility = ["//visibility:public"], diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 36ada1ed9b3ea..b922242dcd10f 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -16,12 +16,10 @@ package model import ( "encoding/json" "fmt" - "math" "sync" "time" "github.com/pingcap/errors" - "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" ) @@ -226,60 +224,6 @@ func (h *HistoryInfo) Clean() { h.MultipleTableInfos = nil } -// DDLReorgMeta is meta info of DDL reorganization. -type DDLReorgMeta struct { - // EndHandle is the last handle of the adding indices table. - // We should only backfill indices in the range [startHandle, EndHandle]. - EndHandle int64 `json:"end_handle"` - - SQLMode mysql.SQLMode `json:"sql_mode"` - Warnings map[errors.ErrorID]*terror.Error `json:"warnings"` - WarningsCount map[errors.ErrorID]int64 `json:"warnings_count"` - Location *TimeZoneLocation `json:"location"` - ReorgTp ReorgType `json:"reorg_tp"` - IsDistReorg bool `json:"is_dist_reorg"` -} - -// ReorgType indicates which process is used for the data reorganization. -type ReorgType int8 - -const ( - // ReorgTypeNone means the backfill task is not started yet. - ReorgTypeNone ReorgType = iota - // ReorgTypeTxn means the index records are backfill with transactions. - // All the index KVs are written through the transaction interface. - // This is the original backfill implementation. - ReorgTypeTxn - // ReorgTypeLitMerge means the index records are backfill with lightning. - // The index KVs are encoded to SST files and imported to the storage directly. - // The incremental index KVs written by DML are redirected to a temporary index. - // After the backfill is finished, the temporary index records are merged back to the original index. - ReorgTypeLitMerge - // ReorgTypeTxnMerge means backfill with transactions and merge incremental changes. - // The backfill index KVs are written through the transaction interface. - // The incremental index KVs written by DML are redirected to a temporary index. - // After the backfill is finished, the temporary index records are merged back to the original index. - ReorgTypeTxnMerge -) - -// NeedMergeProcess means the incremental changes need to be merged. -func (tp ReorgType) NeedMergeProcess() bool { - return tp == ReorgTypeLitMerge || tp == ReorgTypeTxnMerge -} - -// String implements fmt.Stringer interface. -func (tp ReorgType) String() string { - switch tp { - case ReorgTypeTxn: - return "txn" - case ReorgTypeLitMerge: - return "ingest" - case ReorgTypeTxnMerge: - return "txn-merge" - } - return "" -} - // TimeZoneLocation represents a single time zone. type TimeZoneLocation struct { Name string `json:"name"` @@ -302,13 +246,6 @@ func (tz *TimeZoneLocation) GetLocation() (*time.Location, error) { return tz.location, err } -// NewDDLReorgMeta new a DDLReorgMeta. -func NewDDLReorgMeta() *DDLReorgMeta { - return &DDLReorgMeta{ - EndHandle: math.MaxInt64, - } -} - // MultiSchemaInfo keeps some information for multi schema change. type MultiSchemaInfo struct { SubJobs []*SubJob `json:"sub_jobs"` @@ -435,36 +372,6 @@ type JobMeta struct { Priority int `json:"priority"` } -// BackfillMeta is meta info of the backfill job. -type BackfillMeta struct { - IsUnique bool `json:"is_unique"` - EndInclude bool `json:"end_include"` - Error *terror.Error `json:"err"` - - SQLMode mysql.SQLMode `json:"sql_mode"` - Warnings map[errors.ErrorID]*terror.Error `json:"warnings"` - WarningsCount map[errors.ErrorID]int64 `json:"warnings_count"` - Location *TimeZoneLocation `json:"location"` - ReorgTp ReorgType `json:"reorg_tp"` - RowCount int64 `json:"row_count"` - StartKey []byte `json:"start_key"` - EndKey []byte `json:"end_key"` - CurrKey []byte `json:"curr_key"` - *JobMeta `json:"job_meta"` -} - -// Encode encodes BackfillMeta with json format. -func (bm *BackfillMeta) Encode() ([]byte, error) { - b, err := json.Marshal(bm) - return b, errors.Trace(err) -} - -// Decode decodes BackfillMeta from the json buffer. -func (bm *BackfillMeta) Decode(b []byte) error { - err := json.Unmarshal(b, bm) - return errors.Trace(err) -} - // Job is for a DDL operation. type Job struct { ID int64 `json:"id"` diff --git a/parser/model/model.go b/parser/model/model.go index ff710b05c7095..65daa4421c385 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -77,41 +77,6 @@ func (s SchemaState) String() string { } } -// BackfillState is the state used by the backfill-merge process. -type BackfillState byte - -const ( - // BackfillStateInapplicable means the backfill-merge process is not used. - BackfillStateInapplicable BackfillState = iota - // BackfillStateRunning is the state that the backfill process is running. - // In this state, the index's write and delete operations are redirected to a temporary index. - BackfillStateRunning - // BackfillStateReadyToMerge is the state that the temporary index's records are ready to be merged back - // to the origin index. - // In this state, the index's write and delete operations are copied to a temporary index. - // This state is used to make sure that all the TiDB instances are aware of the copy during the merge(BackfillStateMerging). - BackfillStateReadyToMerge - // BackfillStateMerging is the state that the temp index is merging back to the origin index. - // In this state, the index's write and delete operations are copied to a temporary index. - BackfillStateMerging -) - -// String implements fmt.Stringer interface. -func (s BackfillState) String() string { - switch s { - case BackfillStateRunning: - return "backfill state running" - case BackfillStateReadyToMerge: - return "backfill state ready to merge" - case BackfillStateMerging: - return "backfill state merging" - case BackfillStateInapplicable: - return "backfill state inapplicable" - default: - return "backfill state unknown" - } -} - const ( // ColumnInfoVersion0 means the column info version is 0. ColumnInfoVersion0 = uint64(0) diff --git a/parser/model/reorg.go b/parser/model/reorg.go new file mode 100644 index 0000000000000..2f6dea6f914db --- /dev/null +++ b/parser/model/reorg.go @@ -0,0 +1,137 @@ +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package model + +import ( + "encoding/json" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/terror" +) + +// DDLReorgMeta is meta info of DDL reorganization. +type DDLReorgMeta struct { + SQLMode mysql.SQLMode `json:"sql_mode"` + Warnings map[errors.ErrorID]*terror.Error `json:"warnings"` + WarningsCount map[errors.ErrorID]int64 `json:"warnings_count"` + Location *TimeZoneLocation `json:"location"` + ReorgTp ReorgType `json:"reorg_tp"` + IsDistReorg bool `json:"is_dist_reorg"` +} + +// ReorgType indicates which process is used for the data reorganization. +type ReorgType int8 + +const ( + // ReorgTypeNone means the backfill task is not started yet. + ReorgTypeNone ReorgType = iota + // ReorgTypeTxn means the index records are backfill with transactions. + // All the index KVs are written through the transaction interface. + // This is the original backfill implementation. + ReorgTypeTxn + // ReorgTypeLitMerge means the index records are backfill with lightning. + // The index KVs are encoded to SST files and imported to the storage directly. + // The incremental index KVs written by DML are redirected to a temporary index. + // After the backfill is finished, the temporary index records are merged back to the original index. + ReorgTypeLitMerge + // ReorgTypeTxnMerge means backfill with transactions and merge incremental changes. + // The backfill index KVs are written through the transaction interface. + // The incremental index KVs written by DML are redirected to a temporary index. + // After the backfill is finished, the temporary index records are merged back to the original index. + ReorgTypeTxnMerge +) + +// NeedMergeProcess means the incremental changes need to be merged. +func (tp ReorgType) NeedMergeProcess() bool { + return tp == ReorgTypeLitMerge || tp == ReorgTypeTxnMerge +} + +// String implements fmt.Stringer interface. +func (tp ReorgType) String() string { + switch tp { + case ReorgTypeTxn: + return "txn" + case ReorgTypeLitMerge: + return "ingest" + case ReorgTypeTxnMerge: + return "txn-merge" + } + return "" +} + +// BackfillState is the state used by the backfill-merge process. +type BackfillState byte + +const ( + // BackfillStateInapplicable means the backfill-merge process is not used. + BackfillStateInapplicable BackfillState = iota + // BackfillStateRunning is the state that the backfill process is running. + // In this state, the index's write and delete operations are redirected to a temporary index. + BackfillStateRunning + // BackfillStateReadyToMerge is the state that the temporary index's records are ready to be merged back + // to the origin index. + // In this state, the index's write and delete operations are copied to a temporary index. + // This state is used to make sure that all the TiDB instances are aware of the copy during the merge(BackfillStateMerging). + BackfillStateReadyToMerge + // BackfillStateMerging is the state that the temp index is merging back to the origin index. + // In this state, the index's write and delete operations are copied to a temporary index. + BackfillStateMerging +) + +// String implements fmt.Stringer interface. +func (s BackfillState) String() string { + switch s { + case BackfillStateRunning: + return "backfill state running" + case BackfillStateReadyToMerge: + return "backfill state ready to merge" + case BackfillStateMerging: + return "backfill state merging" + case BackfillStateInapplicable: + return "backfill state inapplicable" + default: + return "backfill state unknown" + } +} + +// BackfillMeta is meta info of the backfill job. +type BackfillMeta struct { + IsUnique bool `json:"is_unique"` + EndInclude bool `json:"end_include"` + Error *terror.Error `json:"err"` + + SQLMode mysql.SQLMode `json:"sql_mode"` + Warnings map[errors.ErrorID]*terror.Error `json:"warnings"` + WarningsCount map[errors.ErrorID]int64 `json:"warnings_count"` + Location *TimeZoneLocation `json:"location"` + ReorgTp ReorgType `json:"reorg_tp"` + RowCount int64 `json:"row_count"` + StartKey []byte `json:"start_key"` + EndKey []byte `json:"end_key"` + CurrKey []byte `json:"curr_key"` + *JobMeta `json:"job_meta"` +} + +// Encode encodes BackfillMeta with json format. +func (bm *BackfillMeta) Encode() ([]byte, error) { + b, err := json.Marshal(bm) + return b, errors.Trace(err) +} + +// Decode decodes BackfillMeta from the json buffer. +func (bm *BackfillMeta) Decode(b []byte) error { + err := json.Unmarshal(b, bm) + return errors.Trace(err) +} diff --git a/planner/core/casetest/integration_test.go b/planner/core/casetest/integration_test.go index b82ee72c5df4b..d1cec5800aeee 100644 --- a/planner/core/casetest/integration_test.go +++ b/planner/core/casetest/integration_test.go @@ -3281,10 +3281,10 @@ func TestNullConditionForPrefixIndex(t *testing.T) { ps := []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( - "StreamAgg_18 1.00 root funcs:count(Column#7)->Column#5", - "└─IndexReader_19 1.00 root index:StreamAgg_9", + "StreamAgg_17 1.00 root funcs:count(Column#7)->Column#5", + "└─IndexReader_18 1.00 root index:StreamAgg_9", " └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#7", - " └─IndexRangeScan_17 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo")) + " └─IndexRangeScan_16 99.90 cop[tikv] table:t1, index:idx2(c1, c2) range:[\"0xfff\" -inf,\"0xfff\" +inf], keep order:false, stats:pseudo")) } func TestMppVersion(t *testing.T) { diff --git a/planner/core/casetest/physical_plan_test.go b/planner/core/casetest/physical_plan_test.go index 1023fb0a15510..eaad864e8ce1a 100644 --- a/planner/core/casetest/physical_plan_test.go +++ b/planner/core/casetest/physical_plan_test.go @@ -652,6 +652,36 @@ func TestUnmatchedTableInHint(t *testing.T) { } } +func TestIssue37520(t *testing.T) { + store := testkit.CreateMockStore(t, internal.WithMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int primary key, b int);") + tk.MustExec("create table t2(a int, b int, index ia(a));") + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + + planSuiteData := GetPlanSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + func TestMPPHints(t *testing.T) { store := testkit.CreateMockStore(t, internal.WithMockTiFlash(2)) tk := testkit.NewTestKit(t, store) diff --git a/planner/core/casetest/testdata/integration_suite_out.json b/planner/core/casetest/testdata/integration_suite_out.json index bfbf756c08bfe..ae09e8f61168f 100644 --- a/planner/core/casetest/testdata/integration_suite_out.json +++ b/planner/core/casetest/testdata/integration_suite_out.json @@ -10308,4 +10308,4 @@ } ] } -] \ No newline at end of file +] diff --git a/planner/core/casetest/testdata/join_reorder_suite_out.json b/planner/core/casetest/testdata/join_reorder_suite_out.json index 58336dc72da14..d43d072d434b2 100644 --- a/planner/core/casetest/testdata/join_reorder_suite_out.json +++ b/planner/core/casetest/testdata/join_reorder_suite_out.json @@ -6476,7 +6476,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" ] }, { @@ -6495,7 +6495,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6515,7 +6515,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t1, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6535,7 +6535,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6555,7 +6555,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t1) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6595,7 +6595,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" ] }, { @@ -6613,7 +6613,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6632,7 +6632,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t1, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6651,7 +6651,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6670,7 +6670,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t1) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6712,7 +6712,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" ] }, { @@ -6731,7 +6731,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6751,7 +6751,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t1, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6771,7 +6771,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6791,7 +6791,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t1) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6831,7 +6831,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" ] }, { @@ -6849,7 +6849,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6868,7 +6868,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t1, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6887,7 +6887,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -6906,7 +6906,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t1) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -7426,7 +7426,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" ] }, { @@ -7444,7 +7444,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -7463,7 +7463,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t1, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -7482,7 +7482,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t2) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -7501,7 +7501,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t1) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -8881,7 +8881,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" ] }, { @@ -8900,7 +8900,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -8937,7 +8937,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" ] }, { @@ -8954,7 +8954,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -8996,7 +8996,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" ] }, { @@ -9015,7 +9015,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -9052,7 +9052,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" ] }, { @@ -9069,7 +9069,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, @@ -9438,7 +9438,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" ] }, { @@ -9456,7 +9456,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t2, t3) */. Maybe you can use the table alias name", + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, diff --git a/planner/core/casetest/testdata/plan_suite_in.json b/planner/core/casetest/testdata/plan_suite_in.json index ec93e0c3999da..2d76c71377a64 100644 --- a/planner/core/casetest/testdata/plan_suite_in.json +++ b/planner/core/casetest/testdata/plan_suite_in.json @@ -52,8 +52,8 @@ "select /*+ qb_name(qb, v1), broadcast_join(t1@qb, t2@qb) */ * from v1", // Subquery hint - "SELECT /*+ shuffle_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE */ 1 FROM t t1 WHERE t1.b = t.b);", - "SELECT /*+ broadcast_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE */ 1 FROM t t1 WHERE t1.b = t.b);", + "SELECT /*+ shuffle_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE() */ 1 FROM t t1 WHERE t1.b = t.b);", + "SELECT /*+ broadcast_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE() */ 1 FROM t t1 WHERE t1.b = t.b);", "select * from t t1 where t1.a < (select /*+ MPP_1PHASE_AGG() */ sum(t2.a) from t t2 where t2.b = t1.b);", "select * from t t1 where t1.a < (select /*+ MPP_2PHASE_AGG() */ sum(t2.a) from t t2 where t2.b = t1.b);", @@ -68,6 +68,15 @@ "WITH CTE AS (SELECT /*+ MERGE(), broadcast_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;" ] }, + { + "name": "TestIssue37520", + "cases": [ + "select /*+ inl_join(t1@sel_2) */ a, (select b from t1 where t1.a = t2.b) from t2;", + "select /*+ inl_join(t2) */ a, (select b from t1 where t1.a = t2.b) from t2;", + "select /*+ inl_join(t2@sel_2) */ * from t1 where exists ( select /*+ semi_join_rewrite() */ * from t2 where t1.a = t2.a);", + "select /*+ inl_join(t1) */ * from t1 where exists ( select /*+ semi_join_rewrite() */ * from t2 where t1.a = t2.a);" + ] + }, { "name": "TestHintScope", "cases": [ diff --git a/planner/core/casetest/testdata/plan_suite_out.json b/planner/core/casetest/testdata/plan_suite_out.json index 86ee4c51259b2..ef942eba146e6 100644 --- a/planner/core/casetest/testdata/plan_suite_out.json +++ b/planner/core/casetest/testdata/plan_suite_out.json @@ -554,40 +554,46 @@ "Warn": null }, { - "SQL": "SELECT /*+ shuffle_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE */ 1 FROM t t1 WHERE t1.b = t.b);", + "SQL": "SELECT /*+ shuffle_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE() */ 1 FROM t t1 WHERE t1.b = t.b);", "Plan": [ - "TableReader 7992.00 root MppVersion: 1, data:ExchangeSender", - "└─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7992.00 mpp[tiflash] semi join, equal:[eq(test.t.b, test.t.b)]", - " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.b))", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "TableReader 9990.00 root MppVersion: 1, data:ExchangeSender", + "└─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 9990.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)]", + " ├─Projection(Build) 7992.00 mpp[tiflash] test.t.b", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.b, funcs:firstrow(test.t.b)->test.t.b", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.b, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.b, ", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.b, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], - "Warn": [ - "[planner:1815]There are no matching table names for (t) in optimizer hint /*+ SHUFFLE_JOIN(t) */ or /*+ SHUFFLE_JOIN(t) */. Maybe you can use the table alias name", - "[parser:1064]Optimizer hint syntax error at line 1 column 109 near \"\" " - ] + "Warn": null }, { - "SQL": "SELECT /*+ broadcast_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE */ 1 FROM t t1 WHERE t1.b = t.b);", + "SQL": "SELECT /*+ broadcast_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE() */ 1 FROM t t1 WHERE t1.b = t.b);", "Plan": [ - "TableReader 7992.00 root MppVersion: 1, data:ExchangeSender", - "└─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7992.00 mpp[tiflash] semi join, equal:[eq(test.t.b, test.t.b)]", - " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.b))", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "TableReader 9990.00 root MppVersion: 1, data:ExchangeSender", + "└─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 9990.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 mpp[tiflash] test.t.b", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.b, funcs:firstrow(test.t.b)->test.t.b", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t.b, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.b, ", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], - "Warn": [ - "[planner:1815]There are no matching table names for (t) in optimizer hint /*+ BROADCAST_JOIN(t) */ or /*+ TIDB_BCJ(t) */. Maybe you can use the table alias name", - "[parser:1064]Optimizer hint syntax error at line 1 column 111 near \"\" " - ] + "Warn": null }, { "SQL": "select * from t t1 where t1.a < (select /*+ MPP_1PHASE_AGG() */ sum(t2.a) from t t2 where t2.b = t1.b);", @@ -855,6 +861,63 @@ } ] }, + { + "Name": "TestIssue37520", + "Cases": [ + { + "SQL": "select /*+ inl_join(t1@sel_2) */ a, (select b from t1 where t1.a = t2.b) from t2;", + "Plan": [ + "IndexJoin 12500.00 root left outer join, inner:TableReader, outer key:test.t2.b, inner key:test.t1.a, equal cond:eq(test.t2.b, test.t1.a)", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableRangeScan", + " └─TableRangeScan 10000.00 cop[tikv] table:t1 range: decided by [test.t2.b], keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ inl_join(t2) */ a, (select b from t1 where t1.a = t2.b) from t2;", + "Plan": [ + "HashJoin 12500.00 root left outer join, equal:[eq(test.t2.b, test.t1.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Optimizer Hint /*+ INL_JOIN(t2) */ or /*+ TIDB_INLJ(t2) */ is inapplicable" + ] + }, + { + "SQL": "select /*+ inl_join(t2@sel_2) */ * from t1 where exists ( select /*+ semi_join_rewrite() */ * from t2 where t1.a = t2.a);", + "Plan": [ + "MergeJoin 9990.00 root inner join, left key:test.t1.a, right key:test.t2.a", + "├─StreamAgg(Build) 7992.00 root group by:test.t2.a, funcs:firstrow(test.t2.a)->test.t2.a", + "│ └─IndexReader 7992.00 root index:StreamAgg", + "│ └─StreamAgg 7992.00 cop[tikv] group by:test.t2.a, ", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t2, index:ia(a) keep order:true, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:true, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Optimizer Hint /*+ INL_JOIN(t2) */ or /*+ TIDB_INLJ(t2) */ is inapplicable" + ] + }, + { + "SQL": "select /*+ inl_join(t1) */ * from t1 where exists ( select /*+ semi_join_rewrite() */ * from t2 where t1.a = t2.a);", + "Plan": [ + "IndexJoin 9990.00 root inner join, inner:TableReader, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)", + "├─StreamAgg(Build) 7992.00 root group by:test.t2.a, funcs:firstrow(test.t2.a)->test.t2.a", + "│ └─IndexReader 7992.00 root index:StreamAgg", + "│ └─StreamAgg 7992.00 cop[tikv] group by:test.t2.a, ", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t2, index:ia(a) keep order:true, stats:pseudo", + "└─TableReader(Probe) 7992.00 root data:TableRangeScan", + " └─TableRangeScan 7992.00 cop[tikv] table:t1 range: decided by [test.t2.a], keep order:false, stats:pseudo" + ], + "Warn": null + } + ] + }, { "Name": "TestHintScope", "Cases": [ diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index e283488d7c7bf..8f943a3065f71 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -972,7 +972,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter } } } - var hashPartColName *ast.ColumnName + var hashPartColName *model.CIStr if tblInfo := ds.table.Meta(); canConvertPointGet && tblInfo.GetPartitionInfo() != nil { // We do not build [batch] point get for dynamic table partitions now. This can be optimized. if ds.ctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { @@ -983,7 +983,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter // decided by the current implementation of `BatchPointGetExec::initialize()`, specifically, // the `getPhysID()` function. Once we optimize that part, we can come back and enable // BatchPointGet plan for more cases. - hashPartColName = getHashPartitionColumnName(ds.ctx, tblInfo) + hashPartColName = getHashOrKeyPartitionColumnName(ds.ctx, tblInfo) if hashPartColName == nil { canConvertPointGet = false } @@ -1165,7 +1165,6 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c if prop.TaskTp == property.RootTaskType { cop.indexPlanFinished = true task = cop.convertToRootTask(ds.ctx) - ds.addSelection4PlanCache(task.(*rootTask), ds.tableStats.ScaleByExpectCnt(totalRowCount), prop) } else { task = cop } @@ -1429,20 +1428,6 @@ func (ts *PhysicalTableScan) appendExtraHandleCol(ds *DataSource) (*expression.C return handleCol, true } -// addSelection4PlanCache adds an extra safeguard selection upon this root task for safety. -// When reusing cached plans and rebuilding range for them, the range builder may return an loose range after parameters change. -// When we add the extra selection, it should meet two conditions: -// 1. The length of 'ds.pushedDownConds` should not be zero. -// 2. The result of function `MaybeOverOptimized4PlanCache(ds.pushedDownConds)` call needs to return true. -func (ds *DataSource) addSelection4PlanCache(task *rootTask, stats *property.StatsInfo, prop *property.PhysicalProperty) { - if !expression.MaybeOverOptimized4PlanCache(ds.ctx, ds.pushedDownConds) || len(ds.pushedDownConds) == 0 { - return - } - sel := PhysicalSelection{Conditions: ds.pushedDownConds}.Init(ds.ctx, stats, ds.blockOffset, prop) - sel.SetChildren(task.p) - task.p = sel -} - // convertToIndexScan converts the DataSource to index scan with idx. func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, candidate *candidatePath, _ *physicalOptimizeOp) (task task, err error) { @@ -1539,7 +1524,6 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, is.addPushedDownSelection(cop, ds, path, finalStats) if prop.TaskTp == property.RootTaskType { task = task.convertToRootTask(ds.ctx) - ds.addSelection4PlanCache(task.(*rootTask), finalStats, prop) } else if _, ok := task.(*rootTask); ok { return invalidTask, nil } @@ -2083,9 +2067,6 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid // So HashJoin cannot pushdown to tiflash. But we still want TableScan to run on tiflash. task = mppTask task = task.convertToRootTask(ds.ctx) - if !task.invalid() { - ds.addSelection4PlanCache(task.(*rootTask), ds.stats.ScaleByExpectCnt(prop.ExpectedCnt), prop) - } } } return task, nil @@ -2120,7 +2101,6 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid } if prop.TaskTp == property.RootTaskType { task = task.convertToRootTask(ds.ctx) - ds.addSelection4PlanCache(task.(*rootTask), ds.stats.ScaleByExpectCnt(prop.ExpectedCnt), prop) } else if _, ok := task.(*rootTask); ok { return invalidTask, nil } @@ -2230,7 +2210,7 @@ func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candida } func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, - candidate *candidatePath, hashPartColName *ast.ColumnName, opt *physicalOptimizeOp) (task task) { + candidate *candidatePath, hashPartColName *model.CIStr, opt *physicalOptimizeOp) (task task) { if !prop.IsSortItemEmpty() && !candidate.isMatchProp { return invalidTask } @@ -2272,7 +2252,7 @@ func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, batchPointGetPlan.IndexInfo = candidate.path.Index batchPointGetPlan.IdxCols = candidate.path.IdxCols batchPointGetPlan.IdxColLens = candidate.path.IdxColLens - batchPointGetPlan.PartitionColPos = getHashPartitionColumnPos(candidate.path.Index, hashPartColName) + batchPointGetPlan.PartitionColPos = getColumnPosInIndex(candidate.path.Index, hashPartColName) for _, ran := range candidate.path.Ranges { batchPointGetPlan.IndexValues = append(batchPointGetPlan.IndexValues, ran.LowVal) } diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index a87c9556e65b2..9dc91dd087e30 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -4871,8 +4871,8 @@ func TestPlanCacheForIndexRangeFallback(t *testing.T) { rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() // We don't limit range mem usage when rebuilding ranges for the cached plan. // So ["aaaaaaaaaa","aaaaaaaaaa"], ["bbbbbbbbbb","bbbbbbbbbb"], ["cccccccccc","cccccccccc"], ["dddddddddd","dddddddddd"], ["eeeeeeeeee","eeeeeeeeee"] can still be built even if its mem usage exceeds 1330. - require.True(t, strings.Contains(rows[2][0].(string), "IndexRangeScan")) - require.True(t, strings.Contains(rows[2][4].(string), "range:[\"aaaaaaaaaa\",\"aaaaaaaaaa\"], [\"bbbbbbbbbb\",\"bbbbbbbbbb\"], [\"cccccccccc\",\"cccccccccc\"], [\"dddddddddd\",\"dddddddddd\"], [\"eeeeeeeeee\",\"eeeeeeeeee\"]")) + require.True(t, strings.Contains(rows[1][0].(string), "IndexRangeScan")) + require.True(t, strings.Contains(rows[1][4].(string), "range:[\"aaaaaaaaaa\",\"aaaaaaaaaa\"], [\"bbbbbbbbbb\",\"bbbbbbbbbb\"], [\"cccccccccc\",\"cccccccccc\"], [\"dddddddddd\",\"dddddddddd\"], [\"eeeeeeeeee\",\"eeeeeeeeee\"]")) // Test the plan with range fallback would not be put into cache. tk.MustExec("prepare stmt2 from 'select * from t where a in (?, ?, ?, ?, ?) and b in (?, ?, ?, ?, ?)'") diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index c34e63fc82143..e99103ce0fd6f 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -5300,6 +5300,7 @@ func (b *PlanBuilder) buildApplyWithJoinType(outerPlan, innerPlan LogicalPlan, t for i := outerPlan.Schema().Len(); i < ap.Schema().Len(); i++ { ap.names[i] = types.EmptyName } + ap.LogicalJoin.setPreferredJoinTypeAndOrder(b.TableHints()) return ap } @@ -5341,7 +5342,8 @@ func setIsInApplyForCTE(p LogicalPlan, apSchema *expression.Schema) { } func (b *PlanBuilder) buildMaxOneRow(p LogicalPlan) LogicalPlan { - maxOneRow := LogicalMaxOneRow{}.Init(b.ctx, b.getSelectOffset()) + // The query block of the MaxOneRow operator should be the same as that of its child. + maxOneRow := LogicalMaxOneRow{}.Init(b.ctx, p.SelectBlockOffset()) maxOneRow.SetChildren(p) return maxOneRow } @@ -5377,42 +5379,7 @@ func (b *PlanBuilder) buildSemiJoin(outerPlan, innerPlan LogicalPlan, onConditio } } // Apply forces to choose hash join currently, so don't worry the hints will take effect if the semi join is in one apply. - if b.TableHints() != nil { - hintInfo := b.TableHints() - outerAlias := extractTableAlias(outerPlan, joinPlan.blockOffset) - innerAlias := extractTableAlias(innerPlan, joinPlan.blockOffset) - if hintInfo.ifPreferMergeJoin(outerAlias, innerAlias) { - joinPlan.preferJoinType |= preferMergeJoin - } - if hintInfo.ifPreferHashJoin(outerAlias, innerAlias) { - joinPlan.preferJoinType |= preferHashJoin - } - if hintInfo.ifPreferINLJ(innerAlias) { - joinPlan.preferJoinType = preferRightAsINLJInner - } - if hintInfo.ifPreferINLHJ(innerAlias) { - joinPlan.preferJoinType = preferRightAsINLHJInner - } - if hintInfo.ifPreferINLMJ(innerAlias) { - joinPlan.preferJoinType = preferRightAsINLMJInner - } - if hintInfo.ifPreferHJBuild(outerAlias) { - joinPlan.preferJoinType |= preferLeftAsHJBuild - } - if hintInfo.ifPreferHJBuild(innerAlias) { - joinPlan.preferJoinType |= preferRightAsHJBuild - } - if hintInfo.ifPreferHJProbe(outerAlias) { - joinPlan.preferJoinType |= preferLeftAsHJProbe - } - if hintInfo.ifPreferHJProbe(innerAlias) { - joinPlan.preferJoinType |= preferRightAsHJProbe - } - // If there're multiple join hints, they're conflict. - if bits.OnesCount(joinPlan.preferJoinType) > 1 { - return nil, errors.New("Join hints are conflict, you can only specify one type of join") - } - } + joinPlan.setPreferredJoinTypeAndOrder(b.TableHints()) if forceRewrite { joinPlan.preferJoinType |= preferRewriteSemiJoin b.optFlag |= flagSemiJoinRewrite diff --git a/planner/core/partition_prune.go b/planner/core/partition_prune.go index 3ab266340829d..05334fc2ab722 100644 --- a/planner/core/partition_prune.go +++ b/planner/core/partition_prune.go @@ -30,8 +30,8 @@ func PartitionPruning(ctx sessionctx.Context, tbl table.PartitionedTable, conds s := partitionProcessor{} pi := tbl.Meta().Partition switch pi.Type { - case model.PartitionTypeHash: - return s.pruneHashPartition(ctx, tbl, partitionNames, conds, columns, names) + case model.PartitionTypeHash, model.PartitionTypeKey: + return s.pruneHashOrKeyPartition(ctx, tbl, partitionNames, conds, columns, names) case model.PartitionTypeRange: rangeOr, err := s.pruneRangePartition(ctx, pi, tbl, conds, columns, names) if err != nil { diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index 0e8194099b017..aa0d45961fe80 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -37,10 +37,8 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/kvcache" - "github.com/pingcap/tidb/util/logutil" utilpc "github.com/pingcap/tidb/util/plancache" "github.com/pingcap/tidb/util/ranger" - "go.uber.org/zap" ) func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isNonPrepared bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) error { @@ -202,9 +200,7 @@ func getCachedPointPlan(stmt *ast.Prepared, sessVars *variable.SessionVars, stmt // so you don't need to consider whether prepared.useCache is enabled. plan := stmt.CachedPlan.(Plan) names := stmt.CachedNames.(types.NameSlice) - err := RebuildPlan4CachedPlan(plan) - if err != nil { - logutil.BgLogger().Debug("rebuild range failed", zap.Error(err)) + if !RebuildPlan4CachedPlan(plan) { return nil, nil, false, nil } if metrics.ResettablePlanCacheCounterFortTest { @@ -239,9 +235,7 @@ func getCachedPlan(sctx sessionctx.Context, isNonPrepared bool, cacheKey kvcache return nil, nil, false, nil } } - err := RebuildPlan4CachedPlan(cachedVal.Plan) - if err != nil { - logutil.BgLogger().Debug("rebuild range failed", zap.Error(err)) + if !RebuildPlan4CachedPlan(cachedVal.Plan) { return nil, nil, false, nil } sessVars.FoundInPlanCache = true @@ -309,11 +303,24 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared } // RebuildPlan4CachedPlan will rebuild this plan under current user parameters. -func RebuildPlan4CachedPlan(p Plan) error { +func RebuildPlan4CachedPlan(p Plan) (ok bool) { sc := p.SCtx().GetSessionVars().StmtCtx + if !sc.UseCache { + return false // plan-cache is disabled for this query + } + sc.InPreparedPlanBuilding = true defer func() { sc.InPreparedPlanBuilding = false }() - return rebuildRange(p) + if err := rebuildRange(p); err != nil { + // TODO: log or warn this error. + return false // fail to rebuild ranges + } + if !sc.UseCache { + // in this case, the UseCache flag changes from `true` to `false`, then there must be some + // over-optimized operations were triggered, return `false` for safety here. + return false + } + return true } func updateRange(p PhysicalPlan, ranges ranger.Ranges, rangeInfo string) { @@ -400,26 +407,34 @@ func rebuildRange(p Plan) error { if err != nil { return err } - if len(ranges.Ranges) == 0 || len(ranges.AccessConds) != len(x.AccessConditions) { - return errors.New("failed to rebuild range: the length of the range has changed") + if !isSafeRange(x.AccessConditions, ranges, false, nil) { + return errors.New("rebuild to get an unsafe range") } for i := range x.IndexValues { x.IndexValues[i] = ranges.Ranges[0].LowVal[i] } } else { var pkCol *expression.Column + var unsignedIntHandle bool if x.TblInfo.PKIsHandle { if pkColInfo := x.TblInfo.GetPkColInfo(); pkColInfo != nil { pkCol = expression.ColInfo2Col(x.schema.Columns, pkColInfo) } + if !x.TblInfo.IsCommonHandle { + unsignedIntHandle = true + } } if pkCol != nil { - ranges, _, _, err := ranger.BuildTableRange(x.AccessConditions, x.ctx, pkCol.RetType, 0) + ranges, accessConds, remainingConds, err := ranger.BuildTableRange(x.AccessConditions, x.ctx, pkCol.RetType, 0) if err != nil { return err } - if len(ranges) == 0 { - return errors.New("failed to rebuild range: the length of the range has changed") + if !isSafeRange(x.AccessConditions, &ranger.DetachRangeResult{ + Ranges: ranges, + AccessConds: accessConds, + RemainedConds: remainingConds, + }, unsignedIntHandle, nil) { + return errors.New("rebuild to get an unsafe range") } x.Handle = kv.IntHandle(ranges[0].LowVal[0].GetInt64()) } @@ -461,26 +476,34 @@ func rebuildRange(p Plan) error { if err != nil { return err } - if len(ranges.Ranges) != len(x.IndexValues) || len(ranges.AccessConds) != len(x.AccessConditions) { - return errors.New("failed to rebuild range: the length of the range has changed") + if len(ranges.Ranges) != len(x.IndexValues) || !isSafeRange(x.AccessConditions, ranges, false, nil) { + return errors.New("rebuild to get an unsafe range") } for i := range x.IndexValues { copy(x.IndexValues[i], ranges.Ranges[i].LowVal) } } else { var pkCol *expression.Column + var unsignedIntHandle bool if x.TblInfo.PKIsHandle { if pkColInfo := x.TblInfo.GetPkColInfo(); pkColInfo != nil { pkCol = expression.ColInfo2Col(x.schema.Columns, pkColInfo) } + if !x.TblInfo.IsCommonHandle { + unsignedIntHandle = true + } } if pkCol != nil { - ranges, _, _, err := ranger.BuildTableRange(x.AccessConditions, x.ctx, pkCol.RetType, 0) + ranges, accessConds, remainingConds, err := ranger.BuildTableRange(x.AccessConditions, x.ctx, pkCol.RetType, 0) if err != nil { return err } - if len(ranges) != len(x.Handles) { - return errors.New("failed to rebuild range: the length of the range has changed") + if len(ranges) != len(x.Handles) && !isSafeRange(x.AccessConditions, &ranger.DetachRangeResult{ + Ranges: ranges, + AccessConds: accessConds, + RemainedConds: remainingConds, + }, unsignedIntHandle, nil) { + return errors.New("rebuild to get an unsafe range") } for i := range ranges { x.Handles[i] = kv.IntHandle(ranges[i].LowVal[0].GetInt64()) @@ -590,11 +613,14 @@ func buildRangeForTableScan(sctx sessionctx.Context, ts *PhysicalTableScan) (err if err != nil { return err } - if len(res.AccessConds) != len(ts.AccessCondition) { - return errors.New("rebuild range for cached plan failed") + if !isSafeRange(ts.AccessCondition, res, false, ts.Ranges) { + return errors.New("rebuild to get an unsafe range") } ts.Ranges = res.Ranges } else { + if len(ts.AccessCondition) > 0 { + return errors.New("fail to build ranges, cannot get the primary key column") + } ts.Ranges = ranger.FullRange() } } else { @@ -605,11 +631,22 @@ func buildRangeForTableScan(sctx sessionctx.Context, ts *PhysicalTableScan) (err } } if pkCol != nil { - ts.Ranges, _, _, err = ranger.BuildTableRange(ts.AccessCondition, sctx, pkCol.RetType, 0) + ranges, accessConds, remainingConds, err := ranger.BuildTableRange(ts.AccessCondition, sctx, pkCol.RetType, 0) if err != nil { return err } + if !isSafeRange(ts.AccessCondition, &ranger.DetachRangeResult{ + Ranges: ts.Ranges, + AccessConds: accessConds, + RemainedConds: remainingConds, + }, true, ts.Ranges) { + return errors.New("rebuild to get an unsafe range") + } + ts.Ranges = ranges } else { + if len(ts.AccessCondition) > 0 { + return errors.New("fail to build ranges, cannot get the primary key column") + } ts.Ranges = ranger.FullIntRange(false) } } @@ -617,21 +654,40 @@ func buildRangeForTableScan(sctx sessionctx.Context, ts *PhysicalTableScan) (err } func buildRangeForIndexScan(sctx sessionctx.Context, is *PhysicalIndexScan) (err error) { - if len(is.IdxCols) == 0 { - is.Ranges = ranger.FullRange() - return - } res, err := ranger.DetachCondAndBuildRangeForIndex(sctx, is.AccessCondition, is.IdxCols, is.IdxColLens, 0) if err != nil { return err } - if len(res.AccessConds) != len(is.AccessCondition) { - return errors.New("rebuild range for cached plan failed") + if !isSafeRange(is.AccessCondition, res, false, is.Ranges) { + return errors.New("rebuild to get an unsafe range") } is.Ranges = res.Ranges return } +// checkRebuiltRange checks whether the re-built range is safe. +// To re-use a cached plan, the planner needs to rebuild the access range, but as +// parameters change, some unsafe ranges may occur. +// For example, the first time the planner can build a range `(2, 5)` from `a>2 and a<(?)5`, but if the +// parameter changes to `(?)1`, then it'll get an unsafe range `(empty)`. +// To make plan-cache safer, let the planner abandon the cached plan if it gets an unsafe range here. +func isSafeRange(accessConds []expression.Expression, rebuiltResult *ranger.DetachRangeResult, + unsignedIntHandle bool, originalRange ranger.Ranges) (safe bool) { + if len(rebuiltResult.RemainedConds) > 0 || // the ranger generates some other extra conditions + len(rebuiltResult.AccessConds) != len(accessConds) || // not all access conditions are used + len(rebuiltResult.Ranges) == 0 { // get an empty range + return false + } + + if len(accessConds) > 0 && // if have accessConds, and + ranger.HasFullRange(rebuiltResult.Ranges, unsignedIntHandle) && // get an full range, and + originalRange != nil && !ranger.HasFullRange(originalRange, unsignedIntHandle) { // the original range is not a full range + return false + } + + return true +} + // CheckPreparedPriv checks the privilege of the prepared statement func CheckPreparedPriv(sctx sessionctx.Context, stmt *PlanCacheStmt, is infoschema.InfoSchema) error { if pm := privilege.GetPrivilegeManager(sctx); pm != nil { diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index be75fc72039c0..68b66de4cd89f 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -83,6 +83,39 @@ func TestInitLRUWithSystemVar(t *testing.T) { require.NotNil(t, lru) } +func TestPlanCacheUnsafeRange(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t (a int unsigned, key(a))`) + tk.MustExec(`prepare st from 'select a from t use index(a) where a 50 { // just for safety and reduce memory cost + checker.cacheable = false + checker.reason = "query has more than 50 constants" + } + return in, !checker.cacheable case *ast.TableName: if checker.schema != nil { if isPartitionTable(checker.schema, node) { diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index e80fd32fc3414..1822f71d83b2a 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -1773,6 +1773,19 @@ func getPartitionInfo(ctx sessionctx.Context, tbl *model.TableInfo, pairs []name return &pi.Definitions[pos], i, int(pos), false } } + case model.PartitionTypeKey: + // The key partition table supports FastPlan when it contains only one partition column + 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}) + if err != nil { + return nil, 0, 0, false + } + return &pi.Definitions[pos], i, pos, false + } + } + } case model.PartitionTypeRange: // left range columns partition for future development if len(pi.Columns) == 0 { @@ -1838,11 +1851,18 @@ func getPartitionColumnPos(idx *model.IndexInfo, partitionExpr *tables.Partition return 0, nil } - var partitionName model.CIStr + var partitionColName model.CIStr switch pi.Type { case model.PartitionTypeHash: if col, ok := partitionExpr.OrigExpr.(*ast.ColumnNameExpr); ok { - partitionName = col.Name.Name + partitionColName = col.Name.Name + } else { + return 0, errors.Errorf("unsupported partition type in BatchGet") + } + case model.PartitionTypeKey: + if len(partitionExpr.KeyPartCols) == 1 { + colInfo := findColNameByColID(tbl.Columns, partitionExpr.KeyPartCols[0]) + partitionColName = colInfo.Name } else { return 0, errors.Errorf("unsupported partition type in BatchGet") } @@ -1850,7 +1870,7 @@ func getPartitionColumnPos(idx *model.IndexInfo, partitionExpr *tables.Partition // left range columns partition for future development if col, ok := partitionExpr.Expr.(*expression.Column); ok && len(pi.Columns) == 0 { colInfo := findColNameByColID(tbl.Columns, col) - partitionName = colInfo.Name + partitionColName = colInfo.Name } else { return 0, errors.Errorf("unsupported partition type in BatchGet") } @@ -1858,27 +1878,23 @@ func getPartitionColumnPos(idx *model.IndexInfo, partitionExpr *tables.Partition // left list columns partition for future development if locateExpr, ok := partitionExpr.ForListPruning.LocateExpr.(*expression.Column); ok && partitionExpr.ForListPruning.ColPrunes == nil { colInfo := findColNameByColID(tbl.Columns, locateExpr) - partitionName = colInfo.Name + partitionColName = colInfo.Name } else { return 0, errors.Errorf("unsupported partition type in BatchGet") } } - for i, idxCol := range idx.Columns { - if partitionName.L == idxCol.Name.L { - return i, nil - } - } - panic("unique index must include all partition columns") + return getColumnPosInIndex(idx, &partitionColName), nil } -// getHashPartitionColumnPos gets the hash partition column's position in the unique index. -func getHashPartitionColumnPos(idx *model.IndexInfo, partitionColName *ast.ColumnName) int { - if partitionColName == nil { +// getColumnPosInIndex gets the column's position in the index. +// It is only used to get partition columns postition in unique index so far. +func getColumnPosInIndex(idx *model.IndexInfo, colName *model.CIStr) int { + if colName == nil { return 0 } for i, idxCol := range idx.Columns { - if partitionColName.Name.L == idxCol.Name.L { + if colName.L == idxCol.Name.L { return i } } @@ -1901,12 +1917,12 @@ func getPartitionExpr(ctx sessionctx.Context, tbl *model.TableInfo) *tables.Part return partTable.PartitionExpr() } -func getHashPartitionColumnName(ctx sessionctx.Context, tbl *model.TableInfo) *ast.ColumnName { +func getHashOrKeyPartitionColumnName(ctx sessionctx.Context, tbl *model.TableInfo) *model.CIStr { pi := tbl.GetPartitionInfo() if pi == nil { return nil } - if pi.Type != model.PartitionTypeHash { + if pi.Type != model.PartitionTypeHash && pi.Type != model.PartitionTypeKey { return nil } is := ctx.GetInfoSchema().(infoschema.InfoSchema) @@ -1916,12 +1932,19 @@ func getHashPartitionColumnName(ctx sessionctx.Context, tbl *model.TableInfo) *a } // PartitionExpr don't need columns and names for hash partition. partitionExpr := table.(partitionTable).PartitionExpr() + if pi.Type == model.PartitionTypeKey { + // used to judge whether the key partition contains only one field + if len(pi.Columns) != 1 { + return nil + } + return &pi.Columns[0] + } expr := partitionExpr.OrigExpr col, ok := expr.(*ast.ColumnNameExpr) if !ok { return nil } - return col.Name + return &col.Name.Name } func findColNameByColID(cols []*model.ColumnInfo, col *expression.Column) *model.ColumnInfo { diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index c217cafbdb242..a4050951d88ac 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -668,7 +668,7 @@ func TestPrepareCacheDeferredFunction(t *testing.T) { tk.MustExec("prepare sel1 from 'select id, c1 from t1 where c1 < now(3)'") sql1 := "execute sel1" - expectedPattern := `IndexReader\(Index\(t1.idx1\)\[\[-inf,[0-9]{4}-(0[1-9]|1[0-2])-(0[1-9]|[1-2][0-9]|3[0-1]) (2[0-3]|[01][0-9]):[0-5][0-9]:[0-5][0-9].[0-9][0-9][0-9]\)\]\)->Sel\(\[lt\(test.t1.c1, now\(3\)\)\]\)` + expectedPattern := `IndexReader\(Index\(t1.idx1\)\[\[-inf,[0-9]{4}-(0[1-9]|1[0-2])-(0[1-9]|[1-2][0-9]|3[0-1]) (2[0-3]|[01][0-9]):[0-5][0-9]:[0-5][0-9].[0-9][0-9][0-9]\)\]\)` var cnt [2]float64 var planStr [2]string @@ -1419,7 +1419,7 @@ func TestIssue29303(t *testing.T) { tk.MustQuery(`execute stmt using @a,@b,@c,@d`).Check(testkit.Rows()) tk.MustExec(`set @a="龂", @b="龂", @c="龂", @d="龂"`) tk.MustQuery(`execute stmt using @a,@b,@c,@d`).Check(testkit.Rows("� 龂 � 龂")) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) // unsafe range } func TestIssue34725(t *testing.T) { @@ -1695,7 +1695,7 @@ func TestParamMarker4FastPlan(t *testing.T) { tk.MustQuery("execute stmt using @a2, @a3;").Sort().Check(testkit.Rows("1 7", "1 8", "1 9")) tk.MustExec(`set @a2=4, @a3=2`) tk.MustQuery("execute stmt using @a2, @a3;").Sort().Check(testkit.Rows("1 10", "1 7", "1 8")) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // unsafe range } func TestIssue29565(t *testing.T) { @@ -1990,7 +1990,7 @@ func TestPlanCachePointGetAndTableDual(t *testing.T) { tk.MustQuery("execute s4 using @b4,@a4").Check(testkit.Rows()) tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) tk.MustQuery("execute s4 using @a4,@b4").Check(testkit.Rows("2 1 1")) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } func TestIssue26873(t *testing.T) { diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 6a9b623ceab0f..6bfa27f750e9f 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -123,20 +123,27 @@ func generateHashPartitionExpr(ctx sessionctx.Context, pi *model.PartitionInfo, return exprs[0], nil } -func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl table.Table, partitionNames []model.CIStr, - conds []expression.Expression, columns []*expression.Column, names types.NameSlice) ([]int, []expression.Expression, error) { +func getPartColumnsForHashPartition(ctx sessionctx.Context, hashExpr expression.Expression, + columns []*expression.Column, names types.NameSlice) ([]*expression.Column, []int) { + partCols := expression.ExtractColumns(hashExpr) + colLen := make([]int, 0, len(partCols)) + for i := 0; i < len(partCols); i++ { + partCols[i].Index = i + colLen = append(colLen, types.UnspecifiedLength) + } + return partCols, colLen +} + +func (s *partitionProcessor) getUsedHashPartitions(ctx sessionctx.Context, + tbl table.Table, partitionNames []model.CIStr, columns []*expression.Column, + conds []expression.Expression, names types.NameSlice) ([]int, []expression.Expression, error) { pi := tbl.Meta().Partition - pe, err := generateHashPartitionExpr(ctx, pi, columns, names) + hashExpr, err := generateHashPartitionExpr(ctx, pi, columns, names) if err != nil { return nil, nil, err } - partIdx := expression.ExtractColumns(pe) - colLen := make([]int, 0, len(partIdx)) - for i := 0; i < len(partIdx); i++ { - partIdx[i].Index = i - colLen = append(colLen, types.UnspecifiedLength) - } - detachedResult, err := ranger.DetachCondAndBuildRangeForPartition(ctx, conds, partIdx, colLen, ctx.GetSessionVars().RangeMaxSize) + partCols, colLen := getPartColumnsForHashPartition(ctx, hashExpr, columns, names) + detachedResult, err := ranger.DetachCondAndBuildRangeForPartition(ctx, conds, partCols, colLen, ctx.GetSessionVars().RangeMaxSize) if err != nil { return nil, nil, err } @@ -145,7 +152,7 @@ func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl tabl for _, r := range ranges { if r.IsPointNullable(ctx) { if !r.HighVal[0].IsNull() { - if len(r.HighVal) != len(partIdx) { + if len(r.HighVal) != len(partCols) { used = []int{-1} break } @@ -153,7 +160,7 @@ func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl tabl highLowVals := make([]types.Datum, 0, len(r.HighVal)+len(r.LowVal)) highLowVals = append(highLowVals, r.HighVal...) highLowVals = append(highLowVals, r.LowVal...) - pos, isNull, err := pe.EvalInt(ctx, chunk.MutRowFromDatums(highLowVals).ToRow()) + pos, isNull, err := hashExpr.EvalInt(ctx, chunk.MutRowFromDatums(highLowVals).ToRow()) if err != nil { // If we failed to get the point position, we can just skip and ignore it. continue @@ -171,15 +178,15 @@ func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl tabl // create table t2 (a int, b bigint, index (a), index (b)) partition by hash(a) partitions 10; // desc select * from t2 where t2.a between 10 and 15; // determine whether the partition key is int - if col, ok := pe.(*expression.Column); ok && col.RetType.EvalType() == types.ETInt { + if col, ok := hashExpr.(*expression.Column); ok && col.RetType.EvalType() == types.ETInt { numPartitions := len(pi.Definitions) - posHigh, highIsNull, err := pe.EvalInt(ctx, chunk.MutRowFromDatums(r.HighVal).ToRow()) + posHigh, highIsNull, err := hashExpr.EvalInt(ctx, chunk.MutRowFromDatums(r.HighVal).ToRow()) if err != nil { return nil, nil, err } - posLow, lowIsNull, err := pe.EvalInt(ctx, chunk.MutRowFromDatums(r.LowVal).ToRow()) + posLow, lowIsNull, err := hashExpr.EvalInt(ctx, chunk.MutRowFromDatums(r.LowVal).ToRow()) if err != nil { return nil, nil, err } @@ -231,6 +238,115 @@ func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl tabl break } } + return used, detachedResult.RemainedConds, nil +} + +func (s *partitionProcessor) getUsedKeyPartitions(ctx sessionctx.Context, + tbl table.Table, partitionNames []model.CIStr, columns []*expression.Column, + conds []expression.Expression, names types.NameSlice) ([]int, []expression.Expression, error) { + pi := tbl.Meta().Partition + partExpr := tbl.(partitionTable).PartitionExpr() + partCols, colLen := partExpr.GetPartColumnsForKeyPartition(columns) + detachedResult, err := ranger.DetachCondAndBuildRangeForPartition(ctx, conds, partCols, colLen, ctx.GetSessionVars().RangeMaxSize) + if err != nil { + return nil, nil, err + } + ranges := detachedResult.Ranges + used := make([]int, 0, len(ranges)) + + for _, r := range ranges { + if r.IsPointNullable(ctx) { + if len(r.HighVal) != len(partCols) { + used = []int{FullRange} + break + } + + colVals := make([]types.Datum, 0, len(r.HighVal)) + colVals = append(colVals, r.HighVal...) + idx, err := partExpr.LocateKeyPartition(pi, partCols, colVals) + if err != nil { + // If we failed to get the point position, we can just skip and ignore it. + continue + } + + if len(partitionNames) > 0 && !s.findByName(partitionNames, pi.Definitions[idx].Name.L) { + continue + } + used = append(used, idx) + } else { + if len(partCols) == 1 && partCols[0].RetType.EvalType() == types.ETInt { + col := partCols[0] + posHigh, highIsNull, err := col.EvalInt(ctx, chunk.MutRowFromDatums(r.HighVal).ToRow()) + if err != nil { + return nil, nil, err + } + + posLow, lowIsNull, err := col.EvalInt(ctx, chunk.MutRowFromDatums(r.LowVal).ToRow()) + if err != nil { + return nil, nil, err + } + + // consider whether the range is closed or open + if r.LowExclude { + posLow++ + } + if r.HighExclude { + posHigh-- + } + + var rangeScalar float64 + if mysql.HasUnsignedFlag(col.RetType.GetFlag()) { + rangeScalar = float64(uint64(posHigh)) - float64(uint64(posLow)) // use float64 to avoid integer overflow + } else { + rangeScalar = float64(posHigh) - float64(posLow) // use float64 to avoid integer overflow + } + + // if range is less than the number of partitions, there will be unused partitions we can prune out. + if rangeScalar < float64(pi.Num) && !highIsNull && !lowIsNull { + for i := posLow; i <= posHigh; i++ { + d := types.NewIntDatum(i) + idx, err := partExpr.LocateKeyPartition(pi, partCols, []types.Datum{d}) + if err != nil { + // If we failed to get the point position, we can just skip and ignore it. + continue + } + if len(partitionNames) > 0 && !s.findByName(partitionNames, pi.Definitions[idx].Name.L) { + continue + } + used = append(used, idx) + } + continue + } + } + used = []int{FullRange} + break + } + } + return used, detachedResult.RemainedConds, nil +} + +// getUsedPartitions is used to get used partitions for hash or key partition tables +func (s *partitionProcessor) getUsedPartitions(ctx sessionctx.Context, tbl table.Table, + partitionNames []model.CIStr, columns []*expression.Column, conds []expression.Expression, + names types.NameSlice, partType model.PartitionType) ([]int, []expression.Expression, error) { + if partType == model.PartitionTypeHash { + return s.getUsedHashPartitions(ctx, tbl, partitionNames, columns, conds, names) + } + return s.getUsedKeyPartitions(ctx, tbl, partitionNames, columns, conds, names) +} + +// findUsedPartitions is used to get used partitions for hash or key partition tables. +// The first returning is the used partition index set pruned by `conds`. +// The second returning is the filter conditions which should be kept after pruning. +func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, + tbl table.Table, partitionNames []model.CIStr, conds []expression.Expression, + columns []*expression.Column, names types.NameSlice) ([]int, []expression.Expression, error) { + pi := tbl.Meta().Partition + used, remainedConds, err := s.getUsedPartitions(ctx, tbl, partitionNames, columns, conds, names, pi.Type) + if err != nil { + return nil, nil, err + } + if len(partitionNames) > 0 && len(used) == 1 && used[0] == FullRange { or := partitionRangeOR{partitionRange{0, len(pi.Definitions)}} return s.convertToIntSlice(or, pi, partitionNames), nil, nil @@ -242,7 +358,7 @@ func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl tabl ret = append(ret, used[i]) } } - return ret, detachedResult.RemainedConds, nil + return ret, remainedConds, nil } func (s *partitionProcessor) convertToIntSlice(or partitionRangeOR, pi *model.PartitionInfo, partitionNames []model.CIStr) []int { @@ -274,7 +390,8 @@ func convertToRangeOr(used []int, pi *model.PartitionInfo) partitionRangeOR { return ret } -func (s *partitionProcessor) pruneHashPartition(ctx sessionctx.Context, tbl table.Table, partitionNames []model.CIStr, +// pruneHashOrKeyPartition is used to prune hash or key partition tables +func (s *partitionProcessor) pruneHashOrKeyPartition(ctx sessionctx.Context, tbl table.Table, partitionNames []model.CIStr, conds []expression.Expression, columns []*expression.Column, names types.NameSlice) ([]int, error) { used, _, err := s.findUsedPartitions(ctx, tbl, partitionNames, conds, columns, names) if err != nil { @@ -337,12 +454,13 @@ func (s *partitionProcessor) reconstructTableColNames(ds *DataSource) ([]*types. return names, nil } -func (s *partitionProcessor) processHashPartition(ds *DataSource, pi *model.PartitionInfo, opt *logicalOptimizeOp) (LogicalPlan, error) { +func (s *partitionProcessor) processHashOrKeyPartition(ds *DataSource, pi *model.PartitionInfo, opt *logicalOptimizeOp) (LogicalPlan, error) { names, err := s.reconstructTableColNames(ds) if err != nil { return nil, err } - used, err := s.pruneHashPartition(ds.SCtx(), ds.table, ds.partitionNames, ds.allConds, ds.TblCols, names) + + used, err := s.pruneHashOrKeyPartition(ds.SCtx(), ds.table, ds.partitionNames, ds.allConds, ds.TblCols, names) if err != nil { return nil, err } @@ -644,8 +762,8 @@ func (s *partitionProcessor) prune(ds *DataSource, opt *logicalOptimizeOp) (Logi switch pi.Type { case model.PartitionTypeRange: return s.processRangePartition(ds, pi, opt) - case model.PartitionTypeHash: - return s.processHashPartition(ds, pi, opt) + case model.PartitionTypeHash, model.PartitionTypeKey: + return s.processHashOrKeyPartition(ds, pi, opt) case model.PartitionTypeList: return s.processListPartition(ds, pi, opt) } diff --git a/planner/optimize.go b/planner/optimize.go index 54c98ad16efab..e619aa4fa2df8 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -75,9 +75,11 @@ func matchSQLBinding(sctx sessionctx.Context, stmtNode ast.StmtNode) (bindRecord // getPlanFromNonPreparedPlanCache tries to get an available cached plan from the NonPrepared Plan Cache for this stmt. func getPlanFromNonPreparedPlanCache(ctx context.Context, sctx sessionctx.Context, stmt ast.StmtNode, is infoschema.InfoSchema) (p core.Plan, ns types.NameSlice, ok bool, err error) { + stmtCtx := sctx.GetSessionVars().StmtCtx if !sctx.GetSessionVars().EnableNonPreparedPlanCache || // disabled - sctx.GetSessionVars().StmtCtx.InPreparedPlanBuilding || // already in cached plan rebuilding phase - sctx.GetSessionVars().StmtCtx.InRestrictedSQL { // is internal SQL + stmtCtx.InPreparedPlanBuilding || // already in cached plan rebuilding phase + stmtCtx.EnableOptimizerCETrace || stmtCtx.EnableOptimizeTrace || // in trace + stmtCtx.InRestrictedSQL { // is internal SQL return nil, nil, false, nil } ok, reason := core.NonPreparedPlanCacheableWithCtx(sctx, stmt, is) diff --git a/server/BUILD.bazel b/server/BUILD.bazel index e35d5a9d44ae3..d264f44d576ab 100644 --- a/server/BUILD.bazel +++ b/server/BUILD.bazel @@ -134,6 +134,7 @@ go_test( "conn_stmt_test.go", "conn_test.go", "driver_tidb_test.go", + "extract_test.go", "http_handler_serial_test.go", "http_handler_test.go", "main_test.go", @@ -201,6 +202,7 @@ go_test( "//util/resourcegrouptag", "//util/rowcodec", "//util/sqlexec", + "//util/stmtsummary/v2:stmtsummary", "//util/syncutil", "//util/topsql", "//util/topsql/collector", diff --git a/server/conn.go b/server/conn.go index 9b7117390aebd..8de0dbd1287b7 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1616,23 +1616,29 @@ func (cc *clientConn) writeReq(ctx context.Context, filePath string) error { // handleLoadData does the additional work after processing the 'load data' query. // It sends client a file path, then reads the file content from client, inserts data into database. -func (cc *clientConn) handleLoadData(ctx context.Context, loadDataInfo *executor.LoadDataInfo) error { +func (cc *clientConn) handleLoadData(ctx context.Context, loadDataWorker *executor.LoadDataWorker) error { // If the server handles the load data request, the client has to set the ClientLocalFiles capability. if cc.capability&mysql.ClientLocalFiles == 0 { return errNotAllowedCommand } - if loadDataInfo == nil { + if loadDataWorker == nil { return errors.New("load data info is empty") } - err := cc.writeReq(ctx, loadDataInfo.Path) + err := cc.writeReq(ctx, loadDataWorker.Path) if err != nil { return err } - // use Pipe to convert cc.readPacket to io.Reader - r, w := io.Pipe() + var ( + // use Pipe to convert cc.readPacket to io.Reader + r, w = io.Pipe() + drained bool + wg sync.WaitGroup + ) + wg.Add(1) go func() { + defer wg.Done() //nolint: errcheck defer w.Close() @@ -1649,7 +1655,7 @@ func (cc *clientConn) handleLoadData(ctx context.Context, loadDataInfo *executor } // https://dev.mysql.com/doc/dev/mysql-server/latest/page_protocol_com_query_response_local_infile_request.html if len(data) == 0 { - loadDataInfo.Drained = true + drained = true return } } @@ -1663,16 +1669,19 @@ func (cc *clientConn) handleLoadData(ctx context.Context, loadDataInfo *executor } }() - err = loadDataInfo.Load(ctx, executor.NewSimpleSeekerOnReadCloser(r)) + err = loadDataWorker.Load(ctx, executor.NewSimpleSeekerOnReadCloser(r)) + _ = r.Close() + wg.Wait() + if err != nil { - if !loadDataInfo.Drained { + if !drained { logutil.Logger(ctx).Info("not drained yet, try reading left data from client connection") } // drain the data from client conn util empty packet received, otherwise the connection will be reset // https://dev.mysql.com/doc/dev/mysql-server/latest/page_protocol_com_query_response_local_infile_request.html - for !loadDataInfo.Drained { + for !drained { // check kill flag again, let the draining loop could quit if empty packet could not be received - if atomic.CompareAndSwapUint32(&loadDataInfo.Ctx.GetSessionVars().Killed, 1, 0) { + if atomic.CompareAndSwapUint32(&loadDataWorker.Ctx.GetSessionVars().Killed, 1, 0) { logutil.Logger(ctx).Warn("receiving kill, stop draining data, connection may be reset") return executor.ErrQueryInterrupted } @@ -1682,13 +1691,12 @@ func (cc *clientConn) handleLoadData(ctx context.Context, loadDataInfo *executor break } if len(curData) == 0 { - loadDataInfo.Drained = true + drained = true logutil.Logger(ctx).Info("draining finished for error", zap.Error(err)) break } } } - loadDataInfo.SetMessage() return err } @@ -1886,7 +1894,7 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { } retryable, err = cc.handleStmt(ctx, stmt, parserWarns, i == len(stmts)-1) if err != nil { - action, txnErr := sessiontxn.GetTxnManager(&cc.ctx).OnStmtErrorForNextAction(sessiontxn.StmtErrAfterQuery, err) + action, txnErr := sessiontxn.GetTxnManager(&cc.ctx).OnStmtErrorForNextAction(ctx, sessiontxn.StmtErrAfterQuery, err) if txnErr != nil { err = txnErr break @@ -2083,7 +2091,7 @@ func (cc *clientConn) handleFileTransInConn(ctx context.Context, status uint16) handled = true defer cc.ctx.SetValue(executor.LoadDataVarKey, nil) //nolint:forcetypeassert - if err := cc.handleLoadData(ctx, loadDataInfo.(*executor.LoadDataInfo)); err != nil { + if err := cc.handleLoadData(ctx, loadDataInfo.(*executor.LoadDataWorker)); err != nil { return handled, err } } diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 19e77ce222d51..b6f2716c89631 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -223,7 +223,7 @@ func (cc *clientConn) executePlanCacheStmt(ctx context.Context, stmt interface{} ctx = context.WithValue(ctx, util.ExecDetailsKey, &util.ExecDetails{}) retryable, err := cc.executePreparedStmtAndWriteResult(ctx, stmt.(PreparedStatement), args, useCursor) if err != nil { - action, txnErr := sessiontxn.GetTxnManager(&cc.ctx).OnStmtErrorForNextAction(sessiontxn.StmtErrAfterQuery, err) + action, txnErr := sessiontxn.GetTxnManager(&cc.ctx).OnStmtErrorForNextAction(ctx, sessiontxn.StmtErrAfterQuery, err) if txnErr != nil { return txnErr } diff --git a/server/extract.go b/server/extract.go index 4fd4b46bfe136..30e04c481b4e0 100644 --- a/server/extract.go +++ b/server/extract.go @@ -16,12 +16,17 @@ package server import ( "context" + "fmt" + "io" "net/http" + "os" + "path/filepath" "strconv" "strings" "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/logutil" @@ -48,53 +53,99 @@ func (s *Server) newExtractServeHandler() *ExtractTaskServeHandler { // ServeHTTP serves http func (eh ExtractTaskServeHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { - task, err := buildExtractTask(req) + task, isDump, err := buildExtractTask(req) if err != nil { logutil.BgLogger().Error("build extract task failed", zap.Error(err)) writeError(w, err) return } - _, err = eh.extractHandler.ExtractTask(context.Background(), task) + failpoint.Inject("extractTaskServeHandler", func(val failpoint.Value) { + if val.(bool) { + w.WriteHeader(http.StatusOK) + _, err = w.Write([]byte("mock")) + if err != nil { + writeError(w, err) + } + failpoint.Return() + } + }) + + name, err := eh.extractHandler.ExtractTask(context.Background(), task) if err != nil { logutil.BgLogger().Error("extract task failed", zap.Error(err)) writeError(w, err) return } - // TODO: support return zip file directly for non background job later w.WriteHeader(http.StatusOK) + if !isDump { + _, err = w.Write([]byte(name)) + if err != nil { + logutil.BgLogger().Error("extract handler failed", zap.Error(err)) + } + return + } + content, err := loadExtractResponse(name) + if err != nil { + logutil.BgLogger().Error("load extract task failed", zap.Error(err)) + writeError(w, err) + return + } + _, err = w.Write(content) + if err != nil { + writeError(w, err) + return + } + w.Header().Set("Content-Type", "application/zip") + w.Header().Set("Content-Disposition", fmt.Sprintf("attachment; filename=\"%s.zip\"", name)) } -func buildExtractTask(req *http.Request) (*domain.ExtractTask, error) { +func loadExtractResponse(name string) ([]byte, error) { + path := filepath.Join(domain.GetExtractTaskDirName(), name) + //nolint: gosec + file, err := os.Open(path) + if err != nil { + return nil, err + } + defer file.Close() + content, err := io.ReadAll(file) + if err != nil { + return nil, err + } + return content, nil +} + +func buildExtractTask(req *http.Request) (*domain.ExtractTask, bool, error) { extractTaskType := req.URL.Query().Get(pType) switch strings.ToLower(extractTaskType) { case extractPlanTaskType: return buildExtractPlanTask(req) } logutil.BgLogger().Error("unknown extract task type") - return nil, errors.New("unknown extract task type") + return nil, false, errors.New("unknown extract task type") } -func buildExtractPlanTask(req *http.Request) (*domain.ExtractTask, error) { +func buildExtractPlanTask(req *http.Request) (*domain.ExtractTask, bool, error) { beginStr := req.URL.Query().Get(pBegin) endStr := req.URL.Query().Get(pEnd) begin, err := time.Parse(types.TimeFormat, beginStr) if err != nil { - return nil, err + logutil.BgLogger().Error("extract task begin time failed", zap.Error(err), zap.String("begin", beginStr)) + return nil, false, err } end, err := time.Parse(types.TimeFormat, endStr) if err != nil { - return nil, err + logutil.BgLogger().Error("extract task end time failed", zap.Error(err), zap.String("end", endStr)) + return nil, false, err } - isBackgroundJobStr := req.URL.Query().Get(pIsBackground) - var isBackgroundJob bool - isBackgroundJob, err = strconv.ParseBool(isBackgroundJobStr) + isDumpStr := req.URL.Query().Get(pIsDump) + isDump, err := strconv.ParseBool(isDumpStr) if err != nil { - isBackgroundJob = false + isDump = false } return &domain.ExtractTask{ ExtractType: domain.ExtractPlanType, - IsBackgroundJob: isBackgroundJob, + IsBackgroundJob: false, Begin: begin, End: end, - }, nil + }, isDump, nil } diff --git a/server/extract_test.go b/server/extract_test.go new file mode 100644 index 0000000000000..b9dac858cdc4b --- /dev/null +++ b/server/extract_test.go @@ -0,0 +1,112 @@ +// 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 server + +import ( + "database/sql" + "fmt" + "net/http" + "net/url" + "os" + "testing" + "time" + + "github.com/gorilla/mux" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/types" + stmtsummaryv2 "github.com/pingcap/tidb/util/stmtsummary/v2" + "github.com/stretchr/testify/require" +) + +func TestExtractHandler(t *testing.T) { + setupStmtSummary() + defer closeStmtSummary() + + store := testkit.CreateMockStore(t) + + driver := NewTiDBDriver(store) + client := newTestServerClient() + cfg := newTestConfig() + cfg.Port = client.port + cfg.Status.StatusPort = client.statusPort + cfg.Status.ReportStatus = true + + server, err := NewServer(cfg, driver) + require.NoError(t, err) + defer server.Close() + + client.port = getPortFromTCPAddr(server.listener.Addr()) + client.statusPort = getPortFromTCPAddr(server.statusListener.Addr()) + go func() { + err := server.Run() + require.NoError(t, err) + }() + client.waitUntilServerOnline() + startTime := time.Now() + time.Sleep(time.Second) + prepareData4ExtractPlanTask(t, client) + time.Sleep(time.Second) + endTime := time.Now() + dom, err := session.GetDomain(store) + require.NoError(t, err) + eh := &ExtractTaskServeHandler{extractHandler: dom.GetExtractHandle()} + router := mux.NewRouter() + router.Handle("/extract_task/dump", eh) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/server/extractTaskServeHandler", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/server/extractTaskServeHandler")) + }() + resp0, err := client.fetchStatus(fmt.Sprintf("/extract_task/dump?type=plan&begin=%s&end=%s", + url.QueryEscape(startTime.Format(types.TimeFormat)), url.QueryEscape(endTime.Format(types.TimeFormat)))) + require.NoError(t, err) + defer func() { + require.NoError(t, resp0.Body.Close()) + }() + require.Equal(t, resp0.StatusCode, http.StatusOK) +} + +func prepareData4ExtractPlanTask(t *testing.T, client *testServerClient) { + db, err := sql.Open("mysql", client.getDSN()) + require.NoError(t, err, "Error connecting") + defer func() { + err := db.Close() + require.NoError(t, err) + }() + tk := testkit.NewDBTestKit(t, db) + tk.MustExec("use test") + tk.MustExec("create table t(id int)") + tk.MustExec("select * from t") +} + +func setupStmtSummary() { + stmtsummaryv2.Setup(&stmtsummaryv2.Config{ + Filename: "tidb-statements.log", + }) + config.UpdateGlobal(func(conf *config.Config) { + conf.Instance.StmtSummaryEnablePersistent = true + }) +} + +func closeStmtSummary() { + config.UpdateGlobal(func(conf *config.Config) { + conf.Instance.StmtSummaryEnablePersistent = false + }) + stmtsummaryv2.GlobalStmtSummary.Close() + stmtsummaryv2.GlobalStmtSummary = nil + _ = os.Remove(config.GetGlobalConfig().Instance.StmtSummaryFilename) +} diff --git a/server/http_handler.go b/server/http_handler.go index 5dbf1a9d73d57..4a5eab6c92ffd 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -91,8 +91,8 @@ const ( pDumpPartitionStats = "dumpPartitionStats" pBegin = "begin" pEnd = "end" - pIsBackground = "isBackground" pType = "type" + pIsDump = "isDump" ) // For query string diff --git a/server/server_test.go b/server/server_test.go index 583cee98341fd..4e52453885b27 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -1037,10 +1037,10 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { // can't insert into views (in TiDB) or sequences. issue #20880 _, err = dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table v1", path)) require.Error(t, err) - require.Equal(t, "Error 1105 (HY000): can only load data into base tables", err.Error()) + require.Equal(t, "Error 1288 (HY000): The target table v1 of the LOAD is not updatable", err.Error()) _, err = dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table s1", path)) require.Error(t, err) - require.Equal(t, "Error 1105 (HY000): can only load data into base tables", err.Error()) + require.Equal(t, "Error 1288 (HY000): The target table s1 of the LOAD is not updatable", err.Error()) rs, err1 := dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table test", path)) require.NoError(t, err1) diff --git a/session/txnmanager.go b/session/txnmanager.go index 6916fbf42bb75..1d400ab018f89 100644 --- a/session/txnmanager.go +++ b/session/txnmanager.go @@ -175,21 +175,30 @@ func (m *txnManager) OnStmtStart(ctx context.Context, node ast.StmtNode) error { return m.ctxProvider.OnStmtStart(ctx, m.stmtNode) } -// OnHandlePessimisticStmtStart is the hook that should be called when starts handling a pessimistic DML or +// OnPessimisticStmtStart is the hook that should be called when starts handling a pessimistic DML or // a pessimistic select-for-update statements. -func (m *txnManager) OnHandlePessimisticStmtStart(ctx context.Context) error { +func (m *txnManager) OnPessimisticStmtStart(ctx context.Context) error { if m.ctxProvider == nil { return errors.New("context provider not set") } - return m.ctxProvider.OnHandlePessimisticStmtStart(ctx) + return m.ctxProvider.OnPessimisticStmtStart(ctx) +} + +// OnPessimisticStmtEnd is the hook that should be called when finishes handling a pessimistic DML or +// select-for-update statement. +func (m *txnManager) OnPessimisticStmtEnd(ctx context.Context, isSuccessful bool) error { + if m.ctxProvider == nil { + return errors.New("context provider not set") + } + return m.ctxProvider.OnPessimisticStmtEnd(ctx, isSuccessful) } // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error -func (m *txnManager) OnStmtErrorForNextAction(point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { +func (m *txnManager) OnStmtErrorForNextAction(ctx context.Context, point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { if m.ctxProvider == nil { return sessiontxn.NoIdea() } - return m.ctxProvider.OnStmtErrorForNextAction(point, err) + return m.ctxProvider.OnStmtErrorForNextAction(ctx, point, err) } // ActivateTxn decides to activate txn according to the parameter `active` diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index ed4950f7cd36a..b57594d06082f 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -2392,6 +2392,16 @@ var defaultSysVars = []*SysVar{ s.LoadBasedReplicaReadThreshold = d return nil }}, + {Scope: ScopeGlobal, Name: TiDBTTLRunningTasks, Value: strconv.Itoa(DefTiDBTTLRunningTasks), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency, AllowAutoValue: true, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + val, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + TTLRunningTasks.Store(int32(val)) + return nil + }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { + return strconv.Itoa(int(TTLRunningTasks.Load())), nil + }}, } // FeedbackProbability points to the FeedbackProbability in statistics package. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 7c3a1b736c32c..1a739663f1dfa 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -944,6 +944,9 @@ const ( TiDBStmtSummaryFileMaxSize = "tidb_stmt_summary_file_max_size" // TiDBStmtSummaryFileMaxBackups indicates the maximum number of files written by stmtsummary. TiDBStmtSummaryFileMaxBackups = "tidb_stmt_summary_file_max_backups" + // TiDBTTLRunningTasks limits the count of running ttl tasks. Default to 0, means 3 times the count of TiKV (or no + // limitation, if the storage is not TiKV). + TiDBTTLRunningTasks = "tidb_ttl_running_tasks" ) // TiDB intentional limits @@ -1197,6 +1200,7 @@ const ( DefTiDBTTLDeleteBatchMaxSize = 10240 DefTiDBTTLDeleteBatchMinSize = 1 DefTiDBTTLDeleteRateLimit = 0 + DefTiDBTTLRunningTasks = -1 DefPasswordReuseHistory = 0 DefPasswordReuseTime = 0 DefTiDBStoreBatchSize = 4 @@ -1289,6 +1293,7 @@ var ( HistoricalStatsDuration = atomic.NewDuration(DefTiDBHistoricalStatsDuration) EnableHistoricalStatsForCapture = atomic.NewBool(DefTiDBEnableHistoricalStatsForCapture) EnableResourceControl = atomic.NewBool(DefTiDBEnableResourceControl) + TTLRunningTasks = atomic.NewInt32(DefTiDBTTLRunningTasks) ) var ( diff --git a/sessiontxn/interface.go b/sessiontxn/interface.go index 4c6b50e2e0ae7..e873abf0bf685 100644 --- a/sessiontxn/interface.go +++ b/sessiontxn/interface.go @@ -136,11 +136,14 @@ type TxnContextProvider interface { OnInitialize(ctx context.Context, enterNewTxnType EnterNewTxnType) error // OnStmtStart is the hook that should be called when a new statement started OnStmtStart(ctx context.Context, node ast.StmtNode) error - // OnHandlePessimisticStmtStart is the hook that should be called when starts handling a pessimistic DML or - // a pessimistic select-for-update statements. - OnHandlePessimisticStmtStart(ctx context.Context) error + // OnPessimisticStmtStart is the hook that should be called when starts handling a pessimistic DML or + // a pessimistic select-for-update statement. + OnPessimisticStmtStart(ctx context.Context) error + // OnPessimisticStmtEnd is the hook that should be called when finishes handling a pessimistic DML or + // select-for-update statement. + OnPessimisticStmtEnd(ctx context.Context, isSuccessful bool) error // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error - OnStmtErrorForNextAction(point StmtErrorHandlePoint, err error) (StmtErrorAction, error) + OnStmtErrorForNextAction(ctx context.Context, point StmtErrorHandlePoint, err error) (StmtErrorAction, error) // OnStmtRetry is the hook that should be called when a statement is retried internally. OnStmtRetry(ctx context.Context) error // OnStmtCommit is the hook that should be called when a statement is executed successfully. @@ -185,14 +188,17 @@ type TxnManager interface { OnTxnEnd() // OnStmtStart is the hook that should be called when a new statement started OnStmtStart(ctx context.Context, node ast.StmtNode) error - // OnHandlePessimisticStmtStart is the hook that should be called when starts handling a pessimistic DML or - // a pessimistic select-for-update statements. - OnHandlePessimisticStmtStart(ctx context.Context) error + // OnPessimisticStmtStart is the hook that should be called when starts handling a pessimistic DML or + // a pessimistic select-for-update statement. + OnPessimisticStmtStart(ctx context.Context) error + // OnPessimisticStmtEnd is the hook that should be called when finishes handling a pessimistic DML or + // select-for-update statement. + OnPessimisticStmtEnd(ctx context.Context, isSuccessful bool) error // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error // This method is not required to be called for every error in the statement, // it is only required to be called for some errors handled in some specified points given by the parameter `point`. // When the return error is not nil the return action is 'StmtActionError' and vice versa. - OnStmtErrorForNextAction(point StmtErrorHandlePoint, err error) (StmtErrorAction, error) + OnStmtErrorForNextAction(ctx context.Context, point StmtErrorHandlePoint, err error) (StmtErrorAction, error) // OnStmtRetry is the hook that should be called when a statement retry OnStmtRetry(ctx context.Context) error // OnStmtCommit is the hook that should be called when a statement is executed successfully. diff --git a/sessiontxn/isolation/base.go b/sessiontxn/isolation/base.go index 20aade9941d78..447d54abf73a2 100644 --- a/sessiontxn/isolation/base.go +++ b/sessiontxn/isolation/base.go @@ -207,9 +207,15 @@ func (p *baseTxnContextProvider) OnStmtStart(ctx context.Context, _ ast.StmtNode return nil } -// OnHandlePessimisticStmtStart is the hook that should be called when starts handling a pessimistic DML or +// OnPessimisticStmtStart is the hook that should be called when starts handling a pessimistic DML or // a pessimistic select-for-update statements. -func (p *baseTxnContextProvider) OnHandlePessimisticStmtStart(_ context.Context) error { +func (p *baseTxnContextProvider) OnPessimisticStmtStart(_ context.Context) error { + return nil +} + +// OnPessimisticStmtEnd is the hook that should be called when finishes handling a pessimistic DML or +// select-for-update statement. +func (p *baseTxnContextProvider) OnPessimisticStmtEnd(_ context.Context, _ bool) error { return nil } @@ -241,7 +247,7 @@ func (p *baseTxnContextProvider) OnLocalTemporaryTableCreated() { } // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error -func (p *baseTxnContextProvider) OnStmtErrorForNextAction(point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { +func (p *baseTxnContextProvider) OnStmtErrorForNextAction(ctx context.Context, point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { switch point { case sessiontxn.StmtErrAfterPessimisticLock: // for pessimistic lock error, return the error by default @@ -508,10 +514,10 @@ type basePessimisticTxnContextProvider struct { baseTxnContextProvider } -// OnHandlePessimisticStmtStart is the hook that should be called when starts handling a pessimistic DML or +// OnPessimisticStmtStart is the hook that should be called when starts handling a pessimistic DML or // a pessimistic select-for-update statements. -func (p *basePessimisticTxnContextProvider) OnHandlePessimisticStmtStart(ctx context.Context) error { - if err := p.baseTxnContextProvider.OnHandlePessimisticStmtStart(ctx); err != nil { +func (p *basePessimisticTxnContextProvider) OnPessimisticStmtStart(ctx context.Context) error { + if err := p.baseTxnContextProvider.OnPessimisticStmtStart(ctx); err != nil { return err } if p.sctx.GetSessionVars().PessimisticTransactionAggressiveLocking && @@ -525,38 +531,37 @@ func (p *basePessimisticTxnContextProvider) OnHandlePessimisticStmtStart(ctx con return nil } -// OnStmtRetry is the hook that should be called when a statement is retried internally. -func (p *basePessimisticTxnContextProvider) OnStmtRetry(ctx context.Context) error { - if err := p.baseTxnContextProvider.OnStmtRetry(ctx); err != nil { +// OnPessimisticStmtEnd is the hook that should be called when finishes handling a pessimistic DML or +// select-for-update statement. +func (p *basePessimisticTxnContextProvider) OnPessimisticStmtEnd(ctx context.Context, isSuccessful bool) error { + if err := p.baseTxnContextProvider.OnPessimisticStmtEnd(ctx, isSuccessful); err != nil { return err } if p.txn != nil && p.txn.IsInAggressiveLockingMode() { - if err := p.txn.RetryAggressiveLocking(ctx); err != nil { - return err + if isSuccessful { + if err := p.txn.DoneAggressiveLocking(ctx); err != nil { + return err + } + } else { + if err := p.txn.CancelAggressiveLocking(ctx); err != nil { + return err + } } } return nil } -// OnStmtCommit is the hook that should be called when a statement is executed successfully. -func (p *basePessimisticTxnContextProvider) OnStmtCommit(ctx context.Context) error { - if err := p.baseTxnContextProvider.OnStmtCommit(ctx); err != nil { - return err - } +func (p *basePessimisticTxnContextProvider) retryAggressiveLockingIfNeeded(ctx context.Context) error { if p.txn != nil && p.txn.IsInAggressiveLockingMode() { - if err := p.txn.DoneAggressiveLocking(ctx); err != nil { + if err := p.txn.RetryAggressiveLocking(ctx); err != nil { return err } } return nil } -// OnStmtRollback is the hook that should be called when a statement fails to execute. -func (p *basePessimisticTxnContextProvider) OnStmtRollback(ctx context.Context, isForPessimisticRetry bool) error { - if err := p.baseTxnContextProvider.OnStmtRollback(ctx, isForPessimisticRetry); err != nil { - return err - } - if !isForPessimisticRetry && p.txn != nil && p.txn.IsInAggressiveLockingMode() { +func (p *basePessimisticTxnContextProvider) cancelAggressiveLockingIfNeeded(ctx context.Context) error { + if p.txn != nil && p.txn.IsInAggressiveLockingMode() { if err := p.txn.CancelAggressiveLocking(ctx); err != nil { return err } diff --git a/sessiontxn/isolation/optimistic_test.go b/sessiontxn/isolation/optimistic_test.go index f8042ebe3ab04..2ef474ccbaa76 100644 --- a/sessiontxn/isolation/optimistic_test.go +++ b/sessiontxn/isolation/optimistic_test.go @@ -180,7 +180,7 @@ func TestOptimisticHandleError(t *testing.T) { for _, c := range cases { require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) - action, err := provider.OnStmtErrorForNextAction(c.point, c.err) + action, err := provider.OnStmtErrorForNextAction(context.Background(), c.point, c.err) if c.point == sessiontxn.StmtErrAfterPessimisticLock { require.Error(t, err) require.Same(t, c.err, err) @@ -199,7 +199,7 @@ func TestOptimisticHandleError(t *testing.T) { // OnStmtErrorForNextAction again require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) - action, err = provider.OnStmtErrorForNextAction(c.point, c.err) + action, err = provider.OnStmtErrorForNextAction(context.Background(), c.point, c.err) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionNoIdea, action) diff --git a/sessiontxn/isolation/readcommitted.go b/sessiontxn/isolation/readcommitted.go index ef44834f8bcfb..80cfcc4e4f5d8 100644 --- a/sessiontxn/isolation/readcommitted.go +++ b/sessiontxn/isolation/readcommitted.go @@ -118,14 +118,14 @@ func NeedSetRCCheckTSFlag(ctx sessionctx.Context, node ast.Node) bool { } // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error -func (p *PessimisticRCTxnContextProvider) OnStmtErrorForNextAction(point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { +func (p *PessimisticRCTxnContextProvider) OnStmtErrorForNextAction(ctx context.Context, point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { switch point { case sessiontxn.StmtErrAfterQuery: return p.handleAfterQueryError(err) case sessiontxn.StmtErrAfterPessimisticLock: - return p.handleAfterPessimisticLockError(err) + return p.handleAfterPessimisticLockError(ctx, err) default: - return p.basePessimisticTxnContextProvider.OnStmtErrorForNextAction(point, err) + return p.basePessimisticTxnContextProvider.OnStmtErrorForNextAction(ctx, point, err) } } @@ -215,7 +215,7 @@ func (p *PessimisticRCTxnContextProvider) handleAfterQueryError(queryErr error) return sessiontxn.RetryReady() } -func (p *PessimisticRCTxnContextProvider) handleAfterPessimisticLockError(lockErr error) (sessiontxn.StmtErrorAction, error) { +func (p *PessimisticRCTxnContextProvider) handleAfterPessimisticLockError(ctx context.Context, lockErr error) (sessiontxn.StmtErrorAction, error) { txnCtx := p.sctx.GetSessionVars().TxnCtx retryable := false if deadlock, ok := errors.Cause(lockErr).(*tikverr.ErrDeadlock); ok && deadlock.IsRetryable { @@ -225,6 +225,14 @@ func (p *PessimisticRCTxnContextProvider) handleAfterPessimisticLockError(lockEr zap.Stringer("lockKey", kv.Key(deadlock.LockKey)), zap.Uint64("deadlockKeyHash", deadlock.DeadlockKeyHash)) retryable = true + + // In aggressive locking mode, when statement retry happens, `retryAggressiveLockingIfNeeded` should be + // called to make its state ready for retrying. But single-statement deadlock is an exception. We need to exit + // aggressive locking in single-statement-deadlock case, otherwise the lock this statement has acquired won't be + // released after retrying, so it still blocks another transaction and the deadlock won't be resolved. + if err := p.cancelAggressiveLockingIfNeeded(ctx); err != nil { + return sessiontxn.ErrorAction(err) + } } else if terror.ErrorEqual(kv.ErrWriteConflict, lockErr) { logutil.Logger(p.ctx).Debug("pessimistic write conflict, retry statement", zap.Uint64("txn", txnCtx.StartTS), @@ -237,6 +245,9 @@ func (p *PessimisticRCTxnContextProvider) handleAfterPessimisticLockError(lockEr } if retryable { + if err := p.basePessimisticTxnContextProvider.retryAggressiveLockingIfNeeded(ctx); err != nil { + return sessiontxn.ErrorAction(err) + } return sessiontxn.RetryReady() } return sessiontxn.ErrorAction(lockErr) diff --git a/sessiontxn/isolation/readcommitted_test.go b/sessiontxn/isolation/readcommitted_test.go index cc43396c0bb14..0c727bef3f25a 100644 --- a/sessiontxn/isolation/readcommitted_test.go +++ b/sessiontxn/isolation/readcommitted_test.go @@ -84,7 +84,7 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { require.Equal(t, rcCheckTS, ts) // error will invalidate the rc check - nextAction, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterQuery, kv.ErrWriteConflict) + nextAction, err := provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterQuery, kv.ErrWriteConflict) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionRetryReady, nextAction) compareTS = getOracleTS(t, se) @@ -103,7 +103,7 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { require.Equal(t, rcCheckTS, ts) // other error also invalidate rc check but not retry - nextAction, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterQuery, errors.New("err")) + nextAction, err = provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterQuery, errors.New("err")) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionNoIdea, nextAction) require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) @@ -118,7 +118,7 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { ts, err = provider.GetStmtReadTS() require.NoError(t, err) require.Equal(t, rcCheckTS, ts) - nextAction, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, kv.ErrWriteConflict) + nextAction, err = provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterPessimisticLock, kv.ErrWriteConflict) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionRetryReady, nextAction) compareTS = getOracleTS(t, se) @@ -136,7 +136,7 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { ts, err = provider.GetStmtReadTS() require.NoError(t, err) require.Greater(t, ts, compareTS) - nextAction, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterQuery, kv.ErrWriteConflict) + nextAction, err = provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterQuery, kv.ErrWriteConflict) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionNoIdea, nextAction) } @@ -220,7 +220,7 @@ func TestPessimisticRCTxnContextProviderLockError(t *testing.T) { } { require.NoError(t, executor.ResetContextOfStmt(se, stmt)) require.NoError(t, provider.OnStmtStart(context.TODO(), stmt)) - nextAction, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) + nextAction, err := provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionRetryReady, nextAction) } @@ -232,7 +232,7 @@ func TestPessimisticRCTxnContextProviderLockError(t *testing.T) { } { require.NoError(t, executor.ResetContextOfStmt(se, stmt)) require.NoError(t, provider.OnStmtStart(context.TODO(), stmt)) - nextAction, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) + nextAction, err := provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.Same(t, lockErr, err) require.Equal(t, sessiontxn.StmtActionError, nextAction) } @@ -277,7 +277,7 @@ func TestPessimisticRCTxnContextProviderTS(t *testing.T) { require.Greater(t, readTS, compareTS) // if we should retry, the ts should be updated - nextAction, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, kv.ErrWriteConflict) + nextAction, err := provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterPessimisticLock, kv.ErrWriteConflict) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionRetryReady, nextAction) compareTS = getOracleTS(t, se) diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index 8288ff92bde44..6d74404e3d97d 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -162,10 +162,10 @@ func (p *PessimisticRRTxnContextProvider) OnStmtRetry(ctx context.Context) (err } // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error -func (p *PessimisticRRTxnContextProvider) OnStmtErrorForNextAction(point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { +func (p *PessimisticRRTxnContextProvider) OnStmtErrorForNextAction(ctx context.Context, point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { switch point { case sessiontxn.StmtErrAfterPessimisticLock: - return p.handleAfterPessimisticLockError(err) + return p.handleAfterPessimisticLockError(ctx, err) default: return sessiontxn.NoIdea() } @@ -234,7 +234,7 @@ func notNeedGetLatestTSFromPD(plan plannercore.Plan, inLockOrWriteStmt bool) boo return false } -func (p *PessimisticRRTxnContextProvider) handleAfterPessimisticLockError(lockErr error) (sessiontxn.StmtErrorAction, error) { +func (p *PessimisticRRTxnContextProvider) handleAfterPessimisticLockError(ctx context.Context, lockErr error) (sessiontxn.StmtErrorAction, error) { sessVars := p.sctx.GetSessionVars() txnCtx := sessVars.TxnCtx @@ -248,6 +248,14 @@ func (p *PessimisticRRTxnContextProvider) handleAfterPessimisticLockError(lockEr zap.Uint64("lockTS", deadlock.LockTs), zap.Stringer("lockKey", kv.Key(deadlock.LockKey)), zap.Uint64("deadlockKeyHash", deadlock.DeadlockKeyHash)) + + // In aggressive locking mode, when statement retry happens, `retryAggressiveLockingIfNeeded` should be + // called to make its state ready for retrying. But single-statement deadlock is an exception. We need to exit + // aggressive locking in single-statement-deadlock case, otherwise the lock this statement has acquired won't be + // released after retrying, so it still blocks another transaction and the deadlock won't be resolved. + if err := p.cancelAggressiveLockingIfNeeded(ctx); err != nil { + return sessiontxn.ErrorAction(err) + } } else if terror.ErrorEqual(kv.ErrWriteConflict, lockErr) { // Always update forUpdateTS by getting a new timestamp from PD. // If we use the conflict commitTS as the new forUpdateTS and async commit @@ -281,5 +289,8 @@ func (p *PessimisticRRTxnContextProvider) handleAfterPessimisticLockError(lockEr return sessiontxn.ErrorAction(lockErr) } + if err := p.retryAggressiveLockingIfNeeded(ctx); err != nil { + return sessiontxn.ErrorAction(err) + } return sessiontxn.RetryReady() } diff --git a/sessiontxn/isolation/repeatable_read_test.go b/sessiontxn/isolation/repeatable_read_test.go index a2c0641ff456a..8f95351c06435 100644 --- a/sessiontxn/isolation/repeatable_read_test.go +++ b/sessiontxn/isolation/repeatable_read_test.go @@ -56,7 +56,7 @@ func TestPessimisticRRErrorHandle(t *testing.T) { compareTS := getOracleTS(t, se) lockErr = kv.ErrWriteConflict - nextAction, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) + nextAction, err := provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionRetryReady, nextAction) err = provider.OnStmtRetry(context.TODO()) @@ -72,7 +72,7 @@ func TestPessimisticRRErrorHandle(t *testing.T) { // Update compareTS for the next comparison compareTS = getOracleTS(t, se) lockErr = kv.ErrWriteConflict - nextAction, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) + nextAction, err = provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionRetryReady, nextAction) err = provider.OnStmtStart(context.TODO(), nil) @@ -86,14 +86,14 @@ func TestPessimisticRRErrorHandle(t *testing.T) { require.Greater(t, ts, compareTS2) lockErr = newDeadLockError(false) - nextAction, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) + nextAction, err = provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.Equal(t, lockErr, err) require.Equal(t, sessiontxn.StmtActionError, nextAction) // Update compareTS for the next comparison compareTS = getOracleTS(t, se) lockErr = newDeadLockError(true) - nextAction, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) + nextAction, err = provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionRetryReady, nextAction) err = provider.OnStmtRetry(context.TODO()) @@ -109,7 +109,7 @@ func TestPessimisticRRErrorHandle(t *testing.T) { // Update compareTS for the next comparison compareTS = getOracleTS(t, se) lockErr = newDeadLockError(true) - nextAction, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) + nextAction, err = provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionRetryReady, nextAction) err = provider.OnStmtStart(context.TODO(), nil) @@ -124,13 +124,13 @@ func TestPessimisticRRErrorHandle(t *testing.T) { // StmtErrAfterLock: other errors should only update forUpdateTS but not retry lockErr = errors.New("other error") - nextAction, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) + nextAction, err = provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.Equal(t, lockErr, err) require.Equal(t, sessiontxn.StmtActionError, nextAction) // StmtErrAfterQuery: always not retry and not update forUpdateTS lockErr = kv.ErrWriteConflict - nextAction, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterQuery, lockErr) + nextAction, err = provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterQuery, lockErr) require.Equal(t, sessiontxn.StmtActionNoIdea, nextAction) require.Nil(t, err) } @@ -421,7 +421,7 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { // retry if c.shouldOptimize { - action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, kv.ErrWriteConflict) + action, err = provider.OnStmtErrorForNextAction(context.Background(), sessiontxn.StmtErrAfterPessimisticLock, kv.ErrWriteConflict) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionRetryReady, action) err = provider.OnStmtRetry(context.TODO()) diff --git a/sessiontxn/isolation/serializable.go b/sessiontxn/isolation/serializable.go index 903b1479af79c..77bda169836fa 100644 --- a/sessiontxn/isolation/serializable.go +++ b/sessiontxn/isolation/serializable.go @@ -15,6 +15,8 @@ package isolation import ( + "context" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/sessionctx" @@ -24,24 +26,22 @@ import ( // PessimisticSerializableTxnContextProvider provides txn context for isolation level oracle-like serializable type PessimisticSerializableTxnContextProvider struct { - basePessimisticTxnContextProvider + baseTxnContextProvider } // NewPessimisticSerializableTxnContextProvider returns a new PessimisticSerializableTxnContextProvider func NewPessimisticSerializableTxnContextProvider(sctx sessionctx.Context, causalConsistencyOnly bool) *PessimisticSerializableTxnContextProvider { provider := &PessimisticSerializableTxnContextProvider{ - basePessimisticTxnContextProvider: basePessimisticTxnContextProvider{ - baseTxnContextProvider{ - sctx: sctx, - causalConsistencyOnly: causalConsistencyOnly, - onInitializeTxnCtx: func(txnCtx *variable.TransactionContext) { - txnCtx.IsPessimistic = true - txnCtx.Isolation = ast.Serializable - }, - onTxnActiveFunc: func(txn kv.Transaction, _ sessiontxn.EnterNewTxnType) { - txn.SetOption(kv.Pessimistic, true) - }, + baseTxnContextProvider: baseTxnContextProvider{ + sctx: sctx, + causalConsistencyOnly: causalConsistencyOnly, + onInitializeTxnCtx: func(txnCtx *variable.TransactionContext) { + txnCtx.IsPessimistic = true + txnCtx.Isolation = ast.Serializable + }, + onTxnActiveFunc: func(txn kv.Transaction, _ sessiontxn.EnterNewTxnType) { + txn.SetOption(kv.Pessimistic, true) }, }, } @@ -52,8 +52,7 @@ func NewPessimisticSerializableTxnContextProvider(sctx sessionctx.Context, } // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error -func (p *PessimisticSerializableTxnContextProvider) OnStmtErrorForNextAction( - point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { +func (p *PessimisticSerializableTxnContextProvider) OnStmtErrorForNextAction(ctx context.Context, point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { switch point { case sessiontxn.StmtErrAfterPessimisticLock: // In oracle-like serializable isolation, we do not retry encountering pessimistic lock error. diff --git a/sessiontxn/isolation/serializable_test.go b/sessiontxn/isolation/serializable_test.go index b6086631d0511..85390d447286d 100644 --- a/sessiontxn/isolation/serializable_test.go +++ b/sessiontxn/isolation/serializable_test.go @@ -77,6 +77,7 @@ func TestPessimisticSerializableTxnContextProviderLockError(t *testing.T) { defer tk.MustExec("rollback") se := tk.Session() provider := initializePessimisticSerializableProvider(t, tk) + ctx := context.Background() stmts, _, err := parser.New().Parse("select * from t for update", "", "") require.NoError(t, err) @@ -89,7 +90,7 @@ func TestPessimisticSerializableTxnContextProviderLockError(t *testing.T) { } { require.NoError(t, executor.ResetContextOfStmt(se, stmt)) require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) - nextAction, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) + nextAction, err := provider.OnStmtErrorForNextAction(ctx, sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.Same(t, lockErr, err) require.Equal(t, sessiontxn.StmtActionError, nextAction) } @@ -101,7 +102,7 @@ func TestPessimisticSerializableTxnContextProviderLockError(t *testing.T) { } { require.NoError(t, executor.ResetContextOfStmt(se, stmt)) require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) - nextAction, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) + nextAction, err := provider.OnStmtErrorForNextAction(ctx, sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.Same(t, lockErr, err) require.Equal(t, sessiontxn.StmtActionError, nextAction) } diff --git a/sessiontxn/staleread/provider.go b/sessiontxn/staleread/provider.go index f5d8057882167..a3734202ee6c5 100644 --- a/sessiontxn/staleread/provider.go +++ b/sessiontxn/staleread/provider.go @@ -164,9 +164,15 @@ func (p *StalenessTxnContextProvider) OnStmtStart(ctx context.Context, _ ast.Stm return nil } -// OnHandlePessimisticStmtStart is the hook that should be called when starts handling a pessimistic DML or +// OnPessimisticStmtStart is the hook that should be called when starts handling a pessimistic DML or // a pessimistic select-for-update statements. -func (p *StalenessTxnContextProvider) OnHandlePessimisticStmtStart(_ context.Context) error { +func (p *StalenessTxnContextProvider) OnPessimisticStmtStart(_ context.Context) error { + return nil +} + +// OnPessimisticStmtEnd is the hook that should be called when finishes handling a pessimistic DML or +// select-for-update statement. +func (p *StalenessTxnContextProvider) OnPessimisticStmtEnd(_ context.Context, _ bool) error { return nil } @@ -192,7 +198,7 @@ func (p *StalenessTxnContextProvider) ActivateTxn() (kv.Transaction, error) { } // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error -func (p *StalenessTxnContextProvider) OnStmtErrorForNextAction(_ sessiontxn.StmtErrorHandlePoint, _ error) (sessiontxn.StmtErrorAction, error) { +func (p *StalenessTxnContextProvider) OnStmtErrorForNextAction(ctx context.Context, point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { return sessiontxn.NoIdea() } diff --git a/statistics/selectivity.go b/statistics/selectivity.go index 99458cc04bea6..ff5c6664865ca 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -442,7 +442,7 @@ OUTER: // Try to cover remaining string matching functions by evaluating the expressions with TopN to estimate. if ctx.GetSessionVars().EnableEvalTopNEstimationForStrMatch() { for i, scalarCond := range notCoveredStrMatch { - ok, sel, err := coll.GetSelectivityByFilter(ctx, ctx.GetSessionVars().GetStrMatchDefaultSelectivity(), []expression.Expression{scalarCond}) + ok, sel, err := coll.GetSelectivityByFilter(ctx, []expression.Expression{scalarCond}) if err != nil { sc.AppendWarning(errors.New("Error when using TopN-assisted estimation: " + err.Error())) } @@ -454,7 +454,7 @@ OUTER: delete(notCoveredStrMatch, i) } for i, scalarCond := range notCoveredNegateStrMatch { - ok, sel, err := coll.GetSelectivityByFilter(ctx, ctx.GetSessionVars().GetNegateStrMatchDefaultSelectivity(), []expression.Expression{scalarCond}) + ok, sel, err := coll.GetSelectivityByFilter(ctx, []expression.Expression{scalarCond}) if err != nil { sc.AppendWarning(errors.New("Error when using TopN-assisted estimation: " + err.Error())) } diff --git a/statistics/table.go b/statistics/table.go index 892e964011461..6485ce1f1bec6 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -678,7 +678,7 @@ func CETraceRange(sctx sessionctx.Context, tableID int64, colNames []string, ran func (coll *HistColl) findAvailableStatsForCol(sctx sessionctx.Context, uniqueID int64) (isIndex bool, idx int64) { // try to find available stats in column stats - if colStats, ok := coll.Columns[uniqueID]; ok && colStats != nil && !colStats.IsInvalid(sctx, coll.Pseudo) { + if colStats, ok := coll.Columns[uniqueID]; ok && colStats != nil && !colStats.IsInvalid(sctx, coll.Pseudo) && colStats.IsFullLoad() { return false, uniqueID } // try to find available stats in single column index stats (except for prefix index) @@ -687,7 +687,8 @@ func (coll *HistColl) findAvailableStatsForCol(sctx sessionctx.Context, uniqueID idxStats, ok := coll.Indices[idxStatsIdx] if ok && idxStats.Info.Columns[0].Length == types.UnspecifiedLength && - !idxStats.IsInvalid(coll.Pseudo) { + !idxStats.IsInvalid(coll.Pseudo) && + idxStats.IsFullLoad() { return true, idxStatsIdx } } @@ -695,12 +696,9 @@ func (coll *HistColl) findAvailableStatsForCol(sctx sessionctx.Context, uniqueID return false, -1 } -// GetSelectivityByFilter try to estimate selectivity of expressions by evaluate the expressions using TopN and NULL. -// The data represented by the Histogram would use the defaultSelectivity parameter as the selectivity. +// GetSelectivityByFilter try to estimate selectivity of expressions by evaluate the expressions using TopN, Histogram buckets boundaries and NULL. // Currently, this method can only handle expressions involving a single column. -func (coll *HistColl) GetSelectivityByFilter(sctx sessionctx.Context, - defaultSelectivity float64, - filters []expression.Expression) (ok bool, selectivity float64, err error) { +func (coll *HistColl) GetSelectivityByFilter(sctx sessionctx.Context, filters []expression.Expression) (ok bool, selectivity float64, err error) { // 1. Make sure the expressions // (1) are safe to be evaluated here, // (2) involve only one column, @@ -766,12 +764,13 @@ func (coll *HistColl) GetSelectivityByFilter(sctx sessionctx.Context, // Restore the original Index to avoid unexpected situation. col.Index = originalIndex }() - size := 1 + topNLen := 0 + histBucketsLen := hist.Len() if topn != nil { - size = len(topn.TopN) + topNLen = len(topn.TopN) } - c := chunk.NewChunkWithCapacity([]*types.FieldType{tp}, size) - selected := make([]bool, 0, size) + c := chunk.NewChunkWithCapacity([]*types.FieldType{tp}, mathutil.Max(1, topNLen)) + selected := make([]bool, 0, mathutil.Max(histBucketsLen, topNLen)) // 3. Calculate the TopN part selectivity. // This stage is considered as the core functionality of this method, errors in this stage would make this entire method fail. @@ -797,7 +796,38 @@ func (coll *HistColl) GetSelectivityByFilter(sctx sessionctx.Context, topNSel = float64(topNSelectedCnt) / totalCnt // 4. Calculate the Histogram part selectivity. - histSel = defaultSelectivity * histTotalCnt / totalCnt + // The buckets upper bounds and the Bucket.Repeat are used like the TopN above. + // The buckets lower bounds are used as random samples and are regarded equally. + if hist != nil && histTotalCnt > 0 { + selected = selected[:0] + selected, err = expression.VectorizedFilter(sctx, filters, chunk.NewIterator4Chunk(hist.Bounds), selected) + if err != nil { + return false, 0, err + } + var bucketRepeatTotalCnt, bucketRepeatSelectedCnt, lowerBoundMatchCnt int64 + for i := range hist.Buckets { + bucketRepeatTotalCnt += hist.Buckets[i].Repeat + if len(selected) < 2*i { + // This should not happen, but we add this check for safety. + break + } + if selected[2*i] { + lowerBoundMatchCnt++ + } + if selected[2*i+1] { + bucketRepeatSelectedCnt += hist.Buckets[i].Repeat + } + } + var lowerBoundsRatio, upperBoundsRatio, lowerBoundsSel, upperBoundsSel float64 + upperBoundsRatio = mathutil.Min(float64(bucketRepeatTotalCnt)/histTotalCnt, 1) + lowerBoundsRatio = 1 - upperBoundsRatio + if bucketRepeatTotalCnt > 0 { + upperBoundsSel = float64(bucketRepeatSelectedCnt) / float64(bucketRepeatTotalCnt) + } + lowerBoundsSel = float64(lowerBoundMatchCnt) / float64(histBucketsLen) + histSel = lowerBoundsSel*lowerBoundsRatio + upperBoundsSel*upperBoundsRatio + histSel *= histTotalCnt / totalCnt + } // 5. Calculate the NULL part selectivity. // Errors in this staged would be returned, but would not make this entire method fail. @@ -805,12 +835,10 @@ func (coll *HistColl) GetSelectivityByFilter(sctx sessionctx.Context, c.AppendNull(0) selected = selected[:0] selected, err = expression.VectorizedFilter(sctx, filters, chunk.NewIterator4Chunk(c), selected) - if err != nil || len(selected) != 1 { - nullSel = defaultSelectivity * float64(nullCnt) / totalCnt - } else if selected[0] { - nullSel = float64(nullCnt) / totalCnt - } else { + if err != nil || len(selected) != 1 || !selected[0] { nullSel = 0 + } else { + nullSel = float64(nullCnt) / totalCnt } // 6. Get the final result. diff --git a/statistics/testdata/integration_suite_out.json b/statistics/testdata/integration_suite_out.json index 229e05cd0db2a..4e84b21ea7221 100644 --- a/statistics/testdata/integration_suite_out.json +++ b/statistics/testdata/integration_suite_out.json @@ -355,56 +355,56 @@ { "SQL": "explain format = 'brief' select * from t where a like '%111%'", "Result": [ - "TableReader 30.70 root data:Selection", - "└─Selection 30.70 cop[tikv] like(test.t.a, \"%111%\", 92)", + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.a, \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where a not like '%111%'", "Result": [ - "TableReader 6.30 root data:Selection", - "└─Selection 6.30 cop[tikv] not(like(test.t.a, \"%111%\", 92))", + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.a, \"%111%\", 92))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where a regexp '.*111.*'", "Result": [ - "TableReader 30.70 root data:Selection", - "└─Selection 30.70 cop[tikv] regexp(test.t.a, \".*111.*\")", + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] regexp(test.t.a, \".*111.*\")", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where a not regexp '.*111.*'", "Result": [ - "TableReader 6.30 root data:Selection", - "└─Selection 6.30 cop[tikv] not(istrue_with_null(regexp(test.t.a, \".*111.*\")))", + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(istrue_with_null(regexp(test.t.a, \".*111.*\")))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where ifnull(a, '1111') like '%111%'", "Result": [ - "TableReader 33.70 root data:Selection", - "└─Selection 33.70 cop[tikv] like(ifnull(test.t.a, \"1111\"), \"%111%\", 92)", + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.a, \"1111\"), \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where f like '%111%'", "Result": [ - "TableReader 30.70 root data:Selection", - "└─Selection 30.70 cop[tikv] like(test.t.f, \"%111%\", 92)", + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.f, \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where f not like '%111%'", "Result": [ - "TableReader 6.30 root data:Selection", - "└─Selection 6.30 cop[tikv] not(like(test.t.f, \"%111%\", 92))", + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.f, \"%111%\", 92))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, @@ -427,25 +427,25 @@ { "SQL": "explain format = 'brief' select * from t where ifnull(f, '1111') like '%111%'", "Result": [ - "TableReader 33.70 root data:Selection", - "└─Selection 33.70 cop[tikv] like(ifnull(test.t.f, \"1111\"), \"%111%\", 92)", + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.f, \"1111\"), \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where a like '%111%' and f rlike '.*111.*'", "Result": [ - "Selection 24.56 root regexp(test.t.f, \".*111.*\")", - "└─TableReader 30.70 root data:Selection", - " └─Selection 30.70 cop[tikv] like(test.t.a, \"%111%\", 92)", + "Selection 24.00 root regexp(test.t.f, \".*111.*\")", + "└─TableReader 30.00 root data:Selection", + " └─Selection 30.00 cop[tikv] like(test.t.a, \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where a like '%111%' or f like '%111%'", "Result": [ - "TableReader 37.84 root data:Selection", - "└─Selection 37.84 cop[tikv] or(like(test.t.a, \"%111%\", 92), like(test.t.f, \"%111%\", 92))", + "TableReader 37.50 root data:Selection", + "└─Selection 37.50 cop[tikv] or(like(test.t.a, \"%111%\", 92), like(test.t.f, \"%111%\", 92))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, @@ -492,16 +492,16 @@ { "SQL": "explain format = 'brief' select * from t where b like '%111%' and f like '%111%'", "Result": [ - "TableReader 3.07 root data:Selection", - "└─Selection 3.07 cop[tikv] like(test.t.b, \"%111%\", 92), like(test.t.f, \"%111%\", 92)", + "TableReader 3.00 root data:Selection", + "└─Selection 3.00 cop[tikv] like(test.t.b, \"%111%\", 92), like(test.t.f, \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where b like '%111%' or f like '%111%'", "Result": [ - "TableReader 31.63 root data:Selection", - "└─Selection 31.63 cop[tikv] or(like(test.t.b, \"%111%\", 92), like(test.t.f, \"%111%\", 92))", + "TableReader 31.00 root data:Selection", + "└─Selection 31.00 cop[tikv] or(like(test.t.b, \"%111%\", 92), like(test.t.f, \"%111%\", 92))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, @@ -585,56 +585,56 @@ { "SQL": "explain format = 'brief' select * from t where a like '%111%'", "Result": [ - "TableReader 30.70 root data:Selection", - "└─Selection 30.70 cop[tikv] like(test.t.a, \"%111%\", 92)", + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.a, \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where a not like '%111%'", "Result": [ - "TableReader 6.30 root data:Selection", - "└─Selection 6.30 cop[tikv] not(like(test.t.a, \"%111%\", 92))", + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.a, \"%111%\", 92))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where a regexp '.*111.*'", "Result": [ - "TableReader 30.70 root data:Selection", - "└─Selection 30.70 cop[tikv] regexp(test.t.a, \".*111.*\")", + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] regexp(test.t.a, \".*111.*\")", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where a not regexp '.*111.*'", "Result": [ - "TableReader 6.30 root data:Selection", - "└─Selection 6.30 cop[tikv] not(istrue_with_null(regexp(test.t.a, \".*111.*\")))", + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(istrue_with_null(regexp(test.t.a, \".*111.*\")))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where ifnull(a, '1111') like '%111%'", "Result": [ - "TableReader 33.70 root data:Selection", - "└─Selection 33.70 cop[tikv] like(ifnull(test.t.a, \"1111\"), \"%111%\", 92)", + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.a, \"1111\"), \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where f like '%111%'", "Result": [ - "TableReader 30.70 root data:Selection", - "└─Selection 30.70 cop[tikv] like(test.t.f, \"%111%\", 92)", + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.f, \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where f not like '%111%'", "Result": [ - "TableReader 6.30 root data:Selection", - "└─Selection 6.30 cop[tikv] not(like(test.t.f, \"%111%\", 92))", + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.f, \"%111%\", 92))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, @@ -657,153 +657,153 @@ { "SQL": "explain format = 'brief' select * from t where ifnull(f, '1111') like '%111%'", "Result": [ - "TableReader 33.70 root data:Selection", - "└─Selection 33.70 cop[tikv] like(ifnull(test.t.f, \"1111\"), \"%111%\", 92)", + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.f, \"1111\"), \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where a like '%111%' and f rlike '.*111.*'", "Result": [ - "Selection 24.56 root regexp(test.t.f, \".*111.*\")", - "└─TableReader 30.70 root data:Selection", - " └─Selection 30.70 cop[tikv] like(test.t.a, \"%111%\", 92)", + "Selection 24.00 root regexp(test.t.f, \".*111.*\")", + "└─TableReader 30.00 root data:Selection", + " └─Selection 30.00 cop[tikv] like(test.t.a, \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where a like '%111%' or f like '%111%'", "Result": [ - "TableReader 37.84 root data:Selection", - "└─Selection 37.84 cop[tikv] or(like(test.t.a, \"%111%\", 92), like(test.t.f, \"%111%\", 92))", + "TableReader 37.50 root data:Selection", + "└─Selection 37.50 cop[tikv] or(like(test.t.a, \"%111%\", 92), like(test.t.f, \"%111%\", 92))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where b like '%111%'", "Result": [ - "TableReader 30.70 root data:Selection", - "└─Selection 30.70 cop[tikv] like(test.t.b, \"%111%\", 92)", + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.b, \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where b not like '%111%'", "Result": [ - "TableReader 6.30 root data:Selection", - "└─Selection 6.30 cop[tikv] not(like(test.t.b, \"%111%\", 92))", + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.b, \"%111%\", 92))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where b regexp '.*111.*'", "Result": [ - "TableReader 30.70 root data:Selection", - "└─Selection 30.70 cop[tikv] regexp(test.t.b, \".*111.*\")", + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] regexp(test.t.b, \".*111.*\")", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where b not regexp '.*111.*'", "Result": [ - "TableReader 6.30 root data:Selection", - "└─Selection 6.30 cop[tikv] not(istrue_with_null(regexp(test.t.b, \".*111.*\")))", + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(istrue_with_null(regexp(test.t.b, \".*111.*\")))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where ifnull(b, '1111') like '%111%'", "Result": [ - "TableReader 33.70 root data:Selection", - "└─Selection 33.70 cop[tikv] like(ifnull(test.t.b, \"1111\"), \"%111%\", 92)", + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.b, \"1111\"), \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where b like '%111%' and f like '%111%'", "Result": [ - "TableReader 23.56 root data:Selection", - "└─Selection 23.56 cop[tikv] like(test.t.b, \"%111%\", 92), like(test.t.f, \"%111%\", 92)", + "TableReader 22.50 root data:Selection", + "└─Selection 22.50 cop[tikv] like(test.t.b, \"%111%\", 92), like(test.t.f, \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where b like '%111%' or f like '%111%'", "Result": [ - "TableReader 37.84 root data:Selection", - "└─Selection 37.84 cop[tikv] or(like(test.t.b, \"%111%\", 92), like(test.t.f, \"%111%\", 92))", + "TableReader 37.50 root data:Selection", + "└─Selection 37.50 cop[tikv] or(like(test.t.b, \"%111%\", 92), like(test.t.f, \"%111%\", 92))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where c like '%111%'", "Result": [ - "TableReader 30.70 root data:Selection", - "└─Selection 30.70 cop[tikv] like(test.t.c, \"%111%\", 92)", + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.c, \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where c not like '%111%'", "Result": [ - "TableReader 6.30 root data:Selection", - "└─Selection 6.30 cop[tikv] not(like(test.t.c, \"%111%\", 92))", + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.c, \"%111%\", 92))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where ifnull(c, '1111') like '%111%'", "Result": [ - "TableReader 33.70 root data:Selection", - "└─Selection 33.70 cop[tikv] like(ifnull(test.t.c, \"1111\"), \"%111%\", 92)", + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.c, \"1111\"), \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where d like '%111%'", "Result": [ - "TableReader 30.70 root data:Selection", - "└─Selection 30.70 cop[tikv] like(test.t.d, \"%111%\", 92)", + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.d, \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where d not like '%111%'", "Result": [ - "TableReader 6.30 root data:Selection", - "└─Selection 6.30 cop[tikv] not(like(test.t.d, \"%111%\", 92))", + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.d, \"%111%\", 92))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where ifnull(c, '1111') like '%111%'", "Result": [ - "TableReader 33.70 root data:Selection", - "└─Selection 33.70 cop[tikv] like(ifnull(test.t.c, \"1111\"), \"%111%\", 92)", + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.c, \"1111\"), \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where e like '%111%'", "Result": [ - "TableReader 30.70 root data:Selection", - "└─Selection 30.70 cop[tikv] like(test.t.e, \"%111%\", 92)", + "TableReader 30.00 root data:Selection", + "└─Selection 30.00 cop[tikv] like(test.t.e, \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where e not like '%111%'", "Result": [ - "TableReader 6.30 root data:Selection", - "└─Selection 6.30 cop[tikv] not(like(test.t.e, \"%111%\", 92))", + "TableReader 7.00 root data:Selection", + "└─Selection 7.00 cop[tikv] not(like(test.t.e, \"%111%\", 92))", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'brief' select * from t where ifnull(e, '1111') like '%111%'", "Result": [ - "TableReader 33.70 root data:Selection", - "└─Selection 33.70 cop[tikv] like(ifnull(test.t.e, \"1111\"), \"%111%\", 92)", + "TableReader 33.00 root data:Selection", + "└─Selection 33.00 cop[tikv] like(ifnull(test.t.e, \"1111\"), \"%111%\", 92)", " └─TableFullScan 40.00 cop[tikv] table:t keep order:false" ] } diff --git a/table/tables/partition.go b/table/tables/partition.go index 79cc3a3f361a4..c7969224799ac 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -19,6 +19,7 @@ import ( "context" stderr "errors" "fmt" + "hash/crc32" "sort" "strconv" "strings" @@ -240,6 +241,8 @@ func newPartitionExpr(tblInfo *model.TableInfo, defs []model.PartitionDefinition return generateRangePartitionExpr(ctx, pi, defs, columns, names) case model.PartitionTypeHash: return generateHashPartitionExpr(ctx, pi, columns, names) + case model.PartitionTypeKey: + return generateKeyPartitionExpr(ctx, pi, columns, names) case model.PartitionTypeList: return generateListPartitionExpr(ctx, tblInfo, defs, columns, names) } @@ -254,6 +257,8 @@ type PartitionExpr struct { OrigExpr ast.ExprNode // Expr is the hash partition expression. Expr expression.Expression + // Used in the key partition + *ForKeyPruning // Used in the range pruning process. *ForRangePruning // Used in the range column pruning process. @@ -263,6 +268,48 @@ type PartitionExpr struct { *ForListPruning } +// GetPartColumnsForKeyPartition is used to get partition columns for key partition table +func (pe *PartitionExpr) GetPartColumnsForKeyPartition(columns []*expression.Column) ([]*expression.Column, []int) { + schema := expression.NewSchema(columns...) + partCols := make([]*expression.Column, len(pe.ColumnOffset)) + colLen := make([]int, 0, len(pe.ColumnOffset)) + for i, offset := range pe.ColumnOffset { + partCols[i] = schema.Columns[offset] + partCols[i].Index = i + colLen = append(colLen, partCols[i].RetType.GetFlen()) + } + 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 + return pe.LocateKeyPartition(pi, []*expression.Column{col}, r) +} + +// LocateKeyPartition is the common interface used to locate the destination partition +func (pe *PartitionExpr) LocateKeyPartition(pi *model.PartitionInfo, + cols []*expression.Column, r []types.Datum) (int, error) { + h := crc32.NewIEEE() + for _, col := range cols { + val := r[col.Index] + if val.Kind() == types.KindNull { + h.Write([]byte{0}) + } else { + data, err := val.ToHashKey() + if err != nil { + return 0, err + } + h.Write(data) + } + } + return int(h.Sum32() % uint32(pi.Num)), nil +} + func initEvalBufferType(t *partitionedTable) { hasExtraHandle := false numCols := len(t.Cols()) @@ -325,6 +372,11 @@ func parseSimpleExprWithNames(p *parser.Parser, ctx sessionctx.Context, exprStr return expression.RewriteSimpleExprWithNames(ctx, exprNode, schema, names) } +// ForKeyPruning is used for key partition pruning. +type ForKeyPruning struct { + KeyPartCols []*expression.Column +} + // ForListPruning is used for list partition pruning. type ForListPruning struct { // LocateExpr uses to locate list partition by row. @@ -599,6 +651,21 @@ func rangePartitionExprStrings(pi *model.PartitionInfo) []string { return s } +func generateKeyPartitionExpr(ctx sessionctx.Context, pi *model.PartitionInfo, + columns []*expression.Column, names types.NameSlice) (*PartitionExpr, error) { + ret := &PartitionExpr{ + ForKeyPruning: &ForKeyPruning{}, + } + _, partColumns, offset, err := extractPartitionExprColumns(ctx, pi, columns, names) + if err != nil { + return nil, errors.Trace(err) + } + ret.ColumnOffset = offset + ret.KeyPartCols = partColumns + + return ret, nil +} + func generateRangePartitionExpr(ctx sessionctx.Context, pi *model.PartitionInfo, defs []model.PartitionDefinition, columns []*expression.Column, names types.NameSlice) (*PartitionExpr, error) { // The caller should assure partition info is not nil. @@ -1131,6 +1198,8 @@ func (t *partitionedTable) locatePartitionCommon(ctx sessionctx.Context, pi *mod // Note that only LIST and RANGE supports REORGANIZE PARTITION // TODO: Add support for ADD PARTITION and COALESCE PARTITION for HASH idx, err = t.locateHashPartition(ctx, pi, r) + case model.PartitionTypeKey: + idx, err = t.locateKeyPartition(pi, r) case model.PartitionTypeList: idx, err = t.locateListPartition(ctx, partitionExpr, r) } @@ -1303,6 +1372,11 @@ func (t *partitionedTable) locateHashPartition(ctx sessionctx.Context, pi *model return int(ret), nil } +// TODO: supports linear hashing +func (t *partitionedTable) locateKeyPartition(pi *model.PartitionInfo, r []types.Datum) (int, error) { + return t.partitionExpr.LocateKeyPartition(pi, t.partitionExpr.KeyPartCols, r) +} + // GetPartition returns a Table, which is actually a partition. func (t *partitionedTable) GetPartition(pid int64) table.PhysicalTable { // Attention, can't simply use `return t.partitions[pid]` here. diff --git a/table/tables/partition_test.go b/table/tables/partition_test.go index 0bac493aa7f35..38bc3d0c5ccf7 100644 --- a/table/tables/partition_test.go +++ b/table/tables/partition_test.go @@ -16,6 +16,8 @@ package tables_test import ( "context" + "fmt" + "strconv" "testing" "github.com/pingcap/failpoint" @@ -700,6 +702,1551 @@ func TestIssue31629(t *testing.T) { } } +type compoundSQL struct { + selectSQL string + point bool + batchPoint bool + pruned bool + executeExplain bool + usedPartition []string + notUsedPartition []string + rowCount int +} + +type partTableCase struct { + partitionbySQL string + selectInfo []compoundSQL +} + +func executePartTableCase(t *testing.T, tk *testkit.TestKit, testCases []partTableCase, + createSQL string, insertSQLs []string, dropSQL string) { + for i, testCase := range testCases { + // create table ... partition by key ... + ddlSQL := createSQL + testCase.partitionbySQL + fmt.Println(i, ":", ddlSQL) + executeSQLWrapper(t, tk, ddlSQL) + // insert data + for _, insertsql := range insertSQLs { + executeSQLWrapper(t, tk, insertsql) + } + // execute testcases + for j, selInfo := range testCase.selectInfo { + fmt.Println(j, ":", selInfo.selectSQL) + tk.MustQuery(selInfo.selectSQL).Check(testkit.Rows(strconv.Itoa(selInfo.rowCount))) + if selInfo.executeExplain { + result := tk.MustQuery("EXPLAIN " + selInfo.selectSQL) + if selInfo.point { + result.CheckContain("Point_Get") + } + if selInfo.batchPoint { + result.CheckContain("Batch_Point_Get") + } + if selInfo.pruned { + for _, part := range selInfo.usedPartition { + result.CheckContain(part) + } + for _, part := range selInfo.notUsedPartition { + result.CheckNotContain(part) + } + } + } + } + executeSQLWrapper(t, tk, dropSQL) + } +} + +func executeSQLWrapper(t *testing.T, tk *testkit.TestKit, SQLString string) { + res, err := tk.Exec(SQLString) + if res != nil { + res.Close() + } + require.Nil(t, err) +} + +func TestKeyPartitionTableBasic(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database partitiondb") + defer tk.MustExec("drop database partitiondb") + tk.MustExec("use partitiondb") + testCases := []struct { + createSQL string + dropSQL string + insertSQL string + selectInfo []compoundSQL + }{ + { + createSQL: "CREATE TABLE tkey0 (col1 INT NOT NULL, col2 DATE NOT NULL, col3 INT NOT NULL, col4 INT NOT NULL,UNIQUE KEY (col3)) PARTITION BY KEY(col3) PARTITIONS 4", + insertSQL: "INSERT INTO tkey0 VALUES(1, '2023-02-22', 1, 1), (2, '2023-02-22', 2, 2), (3, '2023-02-22', 3, 3), (4, '2023-02-22', 4, 4)", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey0", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey0 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey0 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey0 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 0, + }, + { + "SELECT count(*) FROM tkey0 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey0 WHERE col3 = 3", + true, false, true, true, []string{"partition:p3"}, []string{"partition:p0", "partition:p1", "partition:p2"}, 1, + }, + { + "SELECT count(*) FROM tkey0 WHERE col3 = 3 or col3 = 4", + false, false, true, true, []string{"partition:p0", "partition:p3"}, []string{"partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey0 WHERE col3 >1 AND col3 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p0", "partition:p2"}, 2, + }, + }, + + dropSQL: "DROP TABLE IF EXISTS tkey0", + }, + { + createSQL: "CREATE TABLE tkey7 (col1 INT NOT NULL, col2 DATE NOT NULL, col3 INT NOT NULL, col4 INT NOT NULL,UNIQUE KEY (col3,col1)) PARTITION BY KEY(col3,col1) PARTITIONS 4", + insertSQL: "INSERT INTO tkey7 VALUES(1, '2023-02-22', 1, 1), (1, '2023-02-22', 2, 1),(2, '2023-02-22', 2, 2), (3, '2023-02-22', 3, 3), (4, '2023-02-22', 4, 4),(4, '2023-02-22', 5, 4)", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey7", + false, false, false, false, []string{}, []string{}, 6, + }, + { + "SELECT count(*) FROM tkey7 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey7 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey7 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey7 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey7 WHERE col3 = 3", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey7 WHERE col3 = 3 and col1 = 3", + true, false, true, true, []string{"partition:p1"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey7 WHERE col3 = 3 or col3 = 4", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey7 WHERE col3 = 3 and col1 = 3 OR col3 = 4 and col1 = 4", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey7 WHERE col1>1 and col3 >1 AND col3 < 4 and col1<3", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey7", + }, + { + createSQL: "CREATE TABLE tkey8 (col1 INT NOT NULL, col2 DATE NOT NULL, col3 INT NOT NULL, col4 INT NOT NULL,PRIMARY KEY (col3,col1)) PARTITION BY KEY(col3,col1) PARTITIONS 4", + insertSQL: "INSERT INTO tkey8 VALUES(1, '2023-02-22', 111, 1), (1, '2023-02-22', 2, 1),(2, '2023-02-22', 218, 2), (3, '2023-02-22', 3, 3), (4, '2023-02-22', 4, 4),(4, '2023-02-22', 5, 4),(5, '2023-02-22', 5, 5),(5, '2023-02-22', 50, 2),(6, '2023-02-22', 62, 2),(60, '2023-02-22', 6, 5),(70, '2023-02-22', 50, 2),(80, '2023-02-22', 62, 2),(100, '2023-02-22', 62, 2),(2000, '2023-02-22', 6, 5),(400, '2023-02-22', 50, 2),(90, '2023-02-22', 62, 2)", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey8", + false, false, false, false, []string{}, []string{}, 16, + }, + { + "SELECT count(*) FROM tkey8 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey8 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 7, + }, + { + "SELECT count(*) FROM tkey8 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey8 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey8 WHERE col3 = 3", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey8 WHERE col3 = 3 and col1 = 3", + true, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey8 WHERE col3 = 3 or col3 = 4", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey8 WHERE col3 = 3 and col1 = 3 OR col3 = 4 and col1 = 4", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey8 WHERE col1>1 and col3 >1 AND col3 < 4 and col1<3", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 0, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey8", + }, + { + createSQL: "CREATE TABLE tkey6 (col1 INT NOT NULL, col2 DATE NOT NULL, col3 VARCHAR(12) NOT NULL, col4 INT NOT NULL,UNIQUE KEY (col3)) PARTITION BY KEY(col3) PARTITIONS 4", + insertSQL: "INSERT INTO tkey6 VALUES(1, '2023-02-22', 'linpin', 1), (2, '2023-02-22', 'zhangsan', 2), (3, '2023-02-22', 'anqila', 3), (4, '2023-02-22', 'xingtian', 4),(1, '2023-02-22', 'renleifeng', 5), (2, '2023-02-22', 'peilin', 2),(1, '2023-02-22', 'abcdeeg', 7), (2, '2023-02-22', 'rpstdfed', 8)", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey6", + false, false, false, false, []string{}, []string{}, 8, + }, + { + "SELECT count(*) FROM tkey6 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey6 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey6 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey6 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey6 WHERE col3 = 'linpin'", + true, false, true, true, []string{"partition:p3"}, []string{"partition:p0", "partition:p1", "partition:p2"}, 1, + }, + { + "SELECT count(*) FROM tkey6 WHERE col3 = 'zhangsan' or col3 = 'linpin'", + true, true, true, true, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey6 WHERE col3 > 'linpin' AND col3 < 'qing'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey6", + }, + { + createSQL: "CREATE TABLE tkey2 (JYRQ INT not null,KHH VARCHAR(12) not null,ZJZH CHAR(14) not null,primary key (JYRQ, KHH, ZJZH))PARTITION BY KEY(KHH) partitions 4", + insertSQL: "INSERT INTO tkey2 VALUES(1,'nanjing','025'),(2,'huaian','0517'),(3,'zhenjiang','0518'),(4,'changzhou','0519'),(5,'wuxi','0511'),(6,'suzhou','0512'),(7,'xuzhou','0513'),(8,'suqian','0513'),(9,'lianyungang','0514'),(10,'yangzhou','0515'),(11,'taizhou','0516'),(12,'nantong','0520'),(13,'yancheng','0521'),(14,'NANJING','025'),(15,'HUAIAN','0527'),(16,'ZHENJIANG','0529'),(17,'CHANGZHOU','0530')", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey2", + false, false, false, false, []string{}, []string{}, 17, + }, + { + "SELECT count(*) FROM tkey2 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey2 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey2 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey2 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 6, + }, + { + "SELECT count(*) FROM tkey2 WHERE KHH = 'huaian'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p0", "partition:p1", "partition:p2"}, 1, + }, + { + "SELECT count(*) FROM tkey2 WHERE KHH = 'huaian' or KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p0", "partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey2 WHERE KHH > 'nanjing' AND KHH < 'suzhou'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 2, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey2", + }, + { + createSQL: "CREATE TABLE tkey5 (JYRQ INT not null,KHH VARCHAR(12) not null,ZJZH CHAR(14) not null,primary key (KHH, JYRQ, ZJZH))PARTITION BY KEY(KHH) partitions 4", + insertSQL: "INSERT INTO tkey5 VALUES(1,'nanjing','025'),(2,'huaian','0517'),(3,'zhenjiang','0518'),(4,'changzhou','0519'),(5,'wuxi','0511'),(6,'suzhou','0512'),(7,'xuzhou','0513'),(8,'suqian','0513'),(9,'lianyungang','0514'),(10,'yangzhou','0515'),(11,'taizhou','0516'),(12,'nantong','0520'),(13,'yancheng','0521'),(14,'NANJING','025'),(15,'HUAIAN','0527'),(16,'ZHENJIANG','0529'),(17,'CHANGZHOU','0530')", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey5", + false, false, false, false, []string{}, []string{}, 17, + }, + { + "SELECT count(*) FROM tkey5 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey5 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey5 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey5 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 6, + }, + { + "SELECT count(*) FROM tkey5 WHERE KHH = 'huaian'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p0", "partition:p1", "partition:p2"}, 1, + }, + { + "SELECT count(*) FROM tkey5 WHERE KHH = 'huaian' or KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p0", "partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey5 WHERE KHH > 'nanjing' AND KHH < 'suzhou'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 2, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey5", + }, + { + createSQL: "CREATE TABLE tkey4 (JYRQ INT not null,KHH VARCHAR(12) not null,ZJZH CHAR(14) not null,primary key (JYRQ, KHH, ZJZH))PARTITION BY KEY(JYRQ, KHH) partitions 4", + insertSQL: "INSERT INTO tkey4 VALUES(1,'nanjing','025'),(2,'huaian','0517'),(3,'zhenjiang','0518'),(4,'changzhou','0519'),(5,'wuxi','0511'),(6,'suzhou','0512'),(7,'xuzhou','0513'),(8,'suqian','0513'),(9,'lianyungang','0514'),(10,'yangzhou','0515'),(11,'taizhou','0516'),(12,'nantong','0520'),(13,'yancheng','0521'),(14,'NANJING','025'),(15,'HUAIAN','0527'),(16,'ZHENJIANG','0529'),(17,'CHANGZHOU','0530'),(1,'beijing','010'),(2,'beijing','010'),(2,'zzzzwuhan','027')", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey4", + false, false, false, false, []string{}, []string{}, 20, + }, + { + "SELECT count(*) FROM tkey4 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 7, + }, + { + "SELECT count(*) FROM tkey4 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey4 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey4 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ = 2", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian' and JYRQ = 2", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian' and JYRQ = 2 or KHH = 'zhenjiang' and JYRQ = 3", + false, false, true, true, []string{"partition:p0", "partition:p1"}, []string{"partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian' and JYRQ = 2 or KHH = 'zhenjiang' and JYRQ = 3 or KHH = 'HUAIAN' and JYRQ = 15", + false, false, true, true, []string{"partition:p0", "partition:p1"}, []string{"partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey4 WHERE KHH = 'huaian' or KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ = 2 OR JYRQ = 3", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ = 2 OR JYRQ = 3 OR JYRQ = 15", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ >6 AND JYRQ < 10", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey4 WHERE JYRQ >6 and KHH>'lianyungang' AND JYRQ < 10 and KHH<'xuzhou'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey4", + }, + { + createSQL: "CREATE TABLE tkey9 (JYRQ INT not null,KHH VARCHAR(12) not null,ZJZH CHAR(14) not null,primary key (JYRQ, KHH, ZJZH))PARTITION BY KEY(JYRQ, KHH, ZJZH) partitions 4", + insertSQL: "INSERT INTO tkey9 VALUES(1,'nanjing','025'),(2,'huaian','0517'),(3,'zhenjiang','0518'),(4,'changzhou','0519'),(5,'wuxi','0511'),(6,'suzhou','0512'),(7,'xuzhou','0513'),(8,'suqian','0513'),(9,'lianyungang','0514'),(10,'yangzhou','0515'),(11,'taizhou','0516'),(12,'nantong','0520'),(13,'yancheng','0521'),(14,'NANJING','025'),(15,'HUAIAN','0527'),(16,'ZHENJIANG','0529'),(17,'CHANGZHOU','0530'),(1,'beijing','010'),(2,'beijing','010'),(2,'zzzzwuhan','027')", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey9", + false, false, false, false, []string{}, []string{}, 20, + }, + { + "SELECT count(*) FROM tkey9 PARTITION(p0)", + false, false, false, false, []string{}, []string{}, 6, + }, + { + "SELECT count(*) FROM tkey9 PARTITION(p1)", + false, false, false, false, []string{}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey9 PARTITION(p2)", + false, false, false, false, []string{}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey9 PARTITION(p3)", + false, false, false, false, []string{}, []string{}, 8, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' and JYRQ = 2 and ZJZH = '0517'", + true, false, true, true, []string{"partition:p0"}, []string{"partition:p3", "partition:p1", "partition:p2"}, 1, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' and JYRQ = 2", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ = 2", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' and JYRQ = 2 and ZJZH='0517' or KHH = 'zhenjiang' and JYRQ = 3 and ZJZH = '0518'", + false, false, true, true, []string{"partition:p3", "partition:p0"}, []string{"partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' and JYRQ = 2 and ZJZH='0517' or KHH = 'zhenjiang' and JYRQ = 3 and ZJZH = '0518' or KHH = 'NANJING' and JYRQ = 14 and ZJZH = '025'", + false, false, true, true, []string{"partition:p0", "partition:p3"}, []string{"partition:p2", "partition:p1"}, 3, + }, + { + "SELECT count(*) FROM tkey9 WHERE KHH = 'huaian' or KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 2, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ = 2 OR JYRQ = 3", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 4, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ = 2 OR JYRQ = 3 OR JYRQ = 15", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ >6 AND JYRQ < 10", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 3, + }, + { + "SELECT count(*) FROM tkey9 WHERE JYRQ = 2 and KHH = 'huaian' OR JYRQ = 3 and KHH = 'zhenjiang'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 2, + }, + }, + dropSQL: "DROP TABLE IF EXISTS tkey9", + }, + } + + for i, testCase := range testCases { + fmt.Println(i, ":", testCase.createSQL) + executeSQLWrapper(t, tk, testCase.createSQL) + executeSQLWrapper(t, tk, testCase.insertSQL) + for j, selInfo := range testCase.selectInfo { + fmt.Println(j, ":", selInfo.selectSQL) + tk.MustQuery(selInfo.selectSQL).Check(testkit.Rows(strconv.Itoa(selInfo.rowCount))) + if selInfo.executeExplain { + result := tk.MustQuery("EXPLAIN " + selInfo.selectSQL) + if selInfo.point { + result.CheckContain("Point_Get") + } + if selInfo.batchPoint { + result.CheckContain("Batch_Point_Get") + } + if selInfo.pruned { + for _, part := range selInfo.usedPartition { + result.CheckContain(part) + } + } + } + } + executeSQLWrapper(t, tk, testCase.dropSQL) + } +} + +func TestKeyPartitionTableAllFeildType(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database partitiondb3") + defer tk.MustExec("drop database partitiondb3") + tk.MustExec("use partitiondb3") + // partition column is numeric family + createSQL := "create table tkey_numeric(\n" + + "id1 BIT(8) not null,\n" + + "id2 TINYINT not null,\n" + + "id3 BOOL not null,\n" + + "id4 SMALLINT not null,\n" + + "id5 MEDIUMINT not null,\n" + + "id6 INT not null,\n" + + "id7 BIGINT not null,\n" + + "id8 DECIMAL(12,4) not null,\n" + + "id9 FLOAT not null,\n" + + "id10 DOUBLE not null,\n" + + "name varchar(20),\n" + + "primary key(id1,id2,id3,id4,id5,id6,id7,id8,id9,id10)\n" + + ")\n" + dropSQL := "drop table tkey_numeric" + insertSQLS := []string{ + "INSERT INTO tkey_numeric VALUES(1,1,0,1,1,1,1,1.1,120.1,367.45,'linpin'),(12,12,12,12,12,12,12,12.1,1220.1,3267.45,'anqila')", + "INSERT INTO tkey_numeric VALUES(0,2,1,2,2,2,2,2.78,16.78,17.25,'ring'),(33,33,33,33,33,33,33,33.78,336.78,37.25,'black')", + "INSERT INTO tkey_numeric VALUES(2,3,1,3,3,3,3,3.78,26.78,417.25,'liudehua'),(22,23,21,23,23,23,23,32.78,26.72,27.15,'chenchen')", + "INSERT INTO tkey_numeric VALUES(3,3,2,4,4,4,4,4.78,46.48,89.35,'guofucheng'), (4,4,4,5,5,5,5,5.78,56.48,59.35,'zhangxuyou')", + "INSERT INTO tkey_numeric VALUES(5,5,5,5,5,5,5,5.78,56.48,59.35,'xietingfeng'),(34,34,34,34,34,34,34,34.78,346.78,34.25,'dongxu')", + "INSERT INTO tkey_numeric VALUES(250,120,120,250,250,258,348,38.78,186.48,719.35,'chenguanxi'),(35,35,35,250,35,35,35,35.78,356.48,35.35,'chenguanxi')", + } + testCases := []partTableCase{ + { + partitionbySQL: "PARTITION BY KEY(id1) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 5, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id1 = 3", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 1, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id1 = 3 or id1 = 4", + false, false, true, true, []string{"partition:p0", "partition:p3"}, []string{"partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id1 >1 AND id1 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p2", "partition:p0"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id2) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id2 = 3", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p0", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id2 = 3 or id2 = 4", + false, false, true, true, []string{"partition:p0", "partition:p3"}, []string{"partition:p1", "partition:p2"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id2 >1 AND id2 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p0", "partition:p2"}, 3, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id3) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p0", "partition:p1", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id3 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id3 = 5 or id3 = 4", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id3 >1 AND id3 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p2", "partition:p0"}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id4) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 5, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id4 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id4 = 5 or id4 = 4", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id4 >1 AND id4 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p0", "partition:p2"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id5) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p3", "partition:p0"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id5 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id5 = 5 or id5 = 4", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id5 >1 AND id5 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p2", "partition:p0"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id6) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id6 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id6 = 5 or id6 = 4", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id6 >1 AND id6 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p2", "partition:p0"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id7) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id7 = 5", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id7 = 5 or id7 = 4", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id7 >1 AND id7 < 4", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p2", "partition:p0"}, 2, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id8) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id8 = 1.1", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p2", "partition:p0", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id8 = 1.1 or id8 = 33.78", + false, false, true, true, []string{"partition:p0", "partition:p1"}, []string{"partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id8 >1 AND id8 < 4", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 3, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id9) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id9 = 46.48", + false, false, true, true, []string{}, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id9 = 46.48 or id9 = 336.78", + false, false, true, true, []string{}, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id9 >45 AND id9 < 47", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id10) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_numeric", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 12, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_numeric PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id10 = 46.48", + false, false, true, true, []string{}, []string{}, 0, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id10 = 46.48 or id10 = 336.78", + false, false, true, true, []string{}, []string{}, 0, + }, + { + "SELECT count(*) FROM tkey_numeric WHERE id10 >366 AND id10 < 368", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + }, + } + executePartTableCase(t, tk, testCases, createSQL, insertSQLS, dropSQL) + + // partition column is date/time family + createSQL2 := "create table tkey_datetime(\n" + + "id1 DATE not null,\n" + + "id2 TIME not null,\n" + + "id3 DATETIME not null,\n" + + "id4 TIMESTAMP not null,\n" + + "id5 YEAR not null,\n" + + "name varchar(20),\n" + + "primary key(id1, id2, id3, id4, id5)\n" + + ")\n" + dropSQL2 := "drop table tkey_datetime" + insertSQLS2 := []string{ + "insert into tkey_datetime values('2012-04-10', '12:12:12', '2012-04-10 12:12:12', '2012-04-10 12:12:12.12345', 2012, 'linpin')", + "insert into tkey_datetime values('2013-05-11', '13:13:13', '2013-05-11 13:13:13', '2013-05-11 13:13:13.43133', 2013, 'minghua')", + "insert into tkey_datetime values('2014-06-12', '14:14:14', '2014-06-12 14:14:14', '2014-06-12 14:14:14.32344', 2014, 'oyangfeng')", + "insert into tkey_datetime values('2015-07-13', '15:15:15', '2015-07-13 15:15:15', '2015-07-13 15:15:15.42544', 2015, 'pengdehuai')", + "insert into tkey_datetime values('2021-08-14', '16:16:16', '2021-08-14 16:16:16', '2021-08-14 16:16:16.18945', 2021, 'shenwanshan')", + "insert into tkey_datetime values('2022-12-23', '23:12:15', '2022-12-23 23:12:15', '2022-12-23 23:12:15.43133', 2022, 'tangchap')", + "insert into tkey_datetime values('2023-01-12', '20:38:14', '2023-01-12 20:38:14', '2023-01-12 20:38:14.32344', 2023, 'xinyu')", + "insert into tkey_datetime values('2018-07-13', '07:15:15', '2018-07-13 07:15:15', '2018-07-13 07:15:15.42544', 2018, 'zongyang')", + "insert into tkey_datetime values('1980-01-30', '00:12:15', '1980-01-30 00:12:15', '1980-01-30 00:12:15.42544', 1980, 'MAYUWEI')", + "insert into tkey_datetime values('1980-03-30', '00:13:15', '1980-03-30 00:13:15', '1980-03-30 00:13:15.42544', 1980, 'maqinwei')", + } + testCases2 := []partTableCase{ + { + partitionbySQL: "PARTITION BY KEY(id1) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_datetime", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 10, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id1 = '2012-04-10'", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id1 = '2012-04-10' or id1 = '2018-07-13'", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id1 >'2012-04-10' AND id1 < '2014-04-10'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id3) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_datetime", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 10, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id3 = '2012-04-10 12:12:12'", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id3 = '2012-04-10 12:12:12' or id3 = '2021-08-14 16:16:16'", + false, false, true, true, []string{"partition:p3", "partition:p1"}, []string{"partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id3 >'2012-04-10 12:12:12' AND id3 < '2014-04-10 12:12:12'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id4) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_datetime", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 10, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 4, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id4 = '2012-04-10 12:12:12'", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id4 = '2012-04-10 12:12:12' or id4 = '2021-08-14 16:16:16'", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p0", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id4 >'2012-04-10 12:12:12' AND id4 < '2014-04-10 12:12:12'", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id5) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_datetime", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 10, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 3, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id5 = 2012", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id5 = 2012 or id5 = 2018", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_datetime WHERE id5 >2012 AND id5 < 2014", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p3", "partition:p0"}, 1, + }, + }, + }, + } + executePartTableCase(t, tk, testCases2, createSQL2, insertSQLS2, dropSQL2) + + // partition column is string family + createSQL3 := "create table tkey_string(\n" + + "id1 CHAR(16) not null,\n" + + "id2 VARCHAR(16) not null,\n" + + "id3 BINARY(16) not null,\n" + + "id4 VARBINARY(16) not null,\n" + + "id5 BLOB not null,\n" + + "id6 TEXT not null,\n" + + "id7 ENUM('x-small', 'small', 'medium', 'large', 'x-large') not null,\n" + + "id8 SET ('a', 'b', 'c', 'd') not null,\n" + + "name varchar(16),\n" + + "primary key(id1, id2, id3, id4, id7, id8)\n" + + ")\n" + dropSQL3 := "drop table tkey_string" + insertSQLS3 := []string{ + "INSERT INTO tkey_string VALUES('huaian','huaian','huaian','huaian','huaian','huaian','x-small','a','linpin')", + "INSERT INTO tkey_string VALUES('nanjing','nanjing','nanjing','nanjing','nanjing','nanjing','small','b','linpin')", + "INSERT INTO tkey_string VALUES('zhenjiang','zhenjiang','zhenjiang','zhenjiang','zhenjiang','zhenjiang','medium','c','linpin')", + "INSERT INTO tkey_string VALUES('suzhou','suzhou','suzhou','suzhou','suzhou','suzhou','large','d','linpin')", + "INSERT INTO tkey_string VALUES('wuxi','wuxi','wuxi','wuxi','wuxi','wuxi','x-large','a','linpin')", + } + testCases3 := []partTableCase{ + { + partitionbySQL: "PARTITION BY KEY(id1) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id1 = 'huaian'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p0", "partition:p2"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id1 = 'huaian' or id1 = 'suzhou'", + false, false, true, true, []string{"partition:p3", "partition:p0"}, []string{"partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id1 >'huaian' AND id1 < 'suzhou'", + false, false, true, true, []string{"partition:p1", "partition:p2", "partition:p0", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id2) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id2 = 'huaian'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id2 = 'huaian' or id2 = 'suzhou'", + false, false, true, true, []string{"partition:p3", "partition:p0"}, []string{"partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id2 >'huaian' AND id2 < 'suzhou'", + false, false, true, true, []string{"partition:p1", "partition:p2", "partition:p0", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id3) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id3 = 0x73757A686F7500000000000000000000", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id3 = 0x73757A686F7500000000000000000000 or id3 = 0x6E616E6A696E67000000000000000000", + false, false, true, true, []string{"partition:p0", "partition:p1"}, []string{"partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id3 >0x67756169616E00000000000000000000 AND id3 < 0x6E616E6A696E67000000000000000000", + false, false, true, true, []string{"partition:p1", "partition:p0", "partition:p2", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id4) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id4 = 0x68756169616E", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p0", "partition:p2"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id4 = 0x68756169616E or id4 = 0x73757A686F75", + false, false, true, true, []string{"partition:p3", "partition:p0"}, []string{"partition:p1", "partition:p2"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id4 >0x73757A686F75 AND id4 < 0x78757869", + false, false, true, true, []string{"partition:p1", "partition:p2", "partition:p0", "partition:p3"}, []string{}, 1, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id7) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id7 = 'x-small'", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string WHERE id7 = 'x-small' or id7 = 'large'", + false, false, true, true, []string{"partition:p0", "partition:p2"}, []string{"partition:p1", "partition:p3"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id7 > 'large' AND id7 < 'x-small'", + false, false, true, true, []string{"partition:p1", "partition:p0", "partition:p3"}, []string{"partition:p2"}, 3, + }, + }, + }, + { + partitionbySQL: "PARTITION BY KEY(id8) partitions 4", + selectInfo: []compoundSQL{ + { + "SELECT count(*) FROM tkey_string", + false, false, true, true, []string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}, []string{}, 5, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p0)", + false, false, true, true, []string{"partition:p0"}, []string{"partition:p1", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p1)", + false, false, true, true, []string{"partition:p1"}, []string{"partition:p0", "partition:p2", "partition:p3"}, 1, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p2)", + false, false, true, true, []string{"partition:p2"}, []string{"partition:p1", "partition:p0", "partition:p3"}, 0, + }, + { + "SELECT count(*) FROM tkey_string PARTITION(p3)", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id8 = 'a'", + false, false, true, true, []string{"partition:p3"}, []string{"partition:p1", "partition:p2", "partition:p0"}, 2, + }, + { + "SELECT count(*) FROM tkey_string WHERE id8 = 'a' or id8 = 'b'", + false, false, true, true, []string{"partition:p1", "partition:p3"}, []string{"partition:p0", "partition:p2"}, 3, + }, + { + "SELECT count(*) FROM tkey_string WHERE id8 > 'a' AND id8 < 'c'", + false, false, true, true, []string{"partition:p1", "partition:p2", "partition:p0", "partition:p3"}, []string{}, 1, + }, + }, + }, + } + executePartTableCase(t, tk, testCases3, createSQL3, insertSQLS3, dropSQL3) +} + +func TestKeyPartitionTableMixed(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database partitiondb2") + defer tk.MustExec("drop database partitiondb2") + tk.MustExec("use partitiondb2") + // SHOW CREATE TABLE + tk.MustExec("CREATE TABLE tkey1 (col1 INT NOT NULL, col2 DATE NOT NULL,col3 INT NOT NULL, col4 INT NOT NULL, UNIQUE KEY (col3))" + + " PARTITION BY KEY(col3)" + + "(PARTITION `p0`," + + "PARTITION `p1`," + + "PARTITION `p2`," + + "PARTITION `p3`)") + tk.MustQuery("show create table tkey1").Check(testkit.Rows("tkey1 CREATE TABLE `tkey1` (\n" + + " `col1` int(11) NOT NULL,\n" + + " `col2` date NOT NULL,\n" + + " `col3` int(11) NOT NULL,\n" + + " `col4` int(11) NOT NULL,\n" + + " UNIQUE KEY `col3` (`col3`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY KEY (`col3`) PARTITIONS 4")) + + // BLOB, JSON don't support key partition + err := tk.ExecToErr("create table tkey_string(\n" + + "id5 BLOB not null,\n" + + "id6 TEXT not null,\n" + + "name varchar(16)\n" + + ") PARTITION BY KEY(id5) partitions 4\n") + require.Error(t, err) + require.Regexp(t, "Field 'id5' is of a not allowed type for this type of partitioning", err) + + // BLOB, JSON don't support key partition + err = tk.ExecToErr("create table tkey_string2(\n" + + "id5 BLOB not null,\n" + + "id6 TEXT not null,\n" + + "name varchar(16)\n" + + ") PARTITION BY KEY(id6) partitions 4\n") + require.Error(t, err) + require.Regexp(t, "Field 'id6' is of a not allowed type for this type of partitioning", err) + + err = tk.ExecToErr("CREATE TABLE tkey_json (c1 JSON) PARTITION BY KEY(c1) partitions 4") + require.Error(t, err) + require.Regexp(t, "Field 'c1' is of a not allowed type for this type of partitioning", err) + + // It doesn't support LINEAR KEY partition + tk.MustExec("CREATE TABLE tkey_linear (col1 INT, col2 CHAR(5), col3 DATE) PARTITION BY LINEAR KEY(col3) PARTITIONS 5") + result := tk.MustQuery("show warnings") + result.CheckContain("LINEAR KEY is not supported, using non-linear KEY instead") + + // It will ignore ALGORITHM=1|2 + tk.MustExec("CREATE TABLE tkey_algorithm1 (col1 INT, col2 CHAR(5), col3 DATE) PARTITION BY KEY ALGORITHM=1 (col3) PARTITIONS 5") + tk.MustExec("CREATE TABLE tkey_algorithm2 (col1 INT, col2 CHAR(5), col3 DATE) PARTITION BY KEY ALGORITHM=2 (col3) PARTITIONS 5") + + err = tk.ExecToErr("CREATE TABLE tkey_algorithm3 (col1 INT, col2 CHAR(5), col3 DATE) PARTITION BY KEY ALGORITHM=3 (col3) PARTITIONS 5") + require.Error(t, err) + require.Regexp(t, "You have an error in your SQL syntax", err) + + // Key partition can't be as subpartition + tk.MustExec("CREATE TABLE tkey_subpartition1 (JYRQ INT not null,KHH VARCHAR(12) not null,ZJZH CHAR(14) not null,primary key (JYRQ, KHH, ZJZH))" + + "PARTITION BY RANGE(JYRQ)\n" + + "SUBPARTITION BY KEY(KHH) SUBPARTITIONS 2 \n" + + "(\n" + + "PARTITION p0 VALUES LESS THAN (8),\n" + + "PARTITION p1 VALUES LESS THAN (16),\n" + + "PARTITION p2 VALUES LESS THAN MAXVALUE\n" + + ")") + result = tk.MustQuery("show warnings") + result.CheckContain("Unsupported partition type RANGE, treat as normal table") + + // It ignores /*!50100 */ format + tk.MustExec("CREATE TABLE tkey10 (`col1` int, `col2` char(5),`col3` date)" + + "/*!50100 PARTITION BY KEY (col3) PARTITIONS 5 */") + result = tk.MustQuery("show create table tkey10") + result.Check(testkit.Rows("tkey10 CREATE TABLE `tkey10` (\n" + + " `col1` int(11) DEFAULT NULL,\n" + + " `col2` char(5) DEFAULT NULL,\n" + + " `col3` date DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY KEY (`col3`) PARTITIONS 5")) + + // It ignores /*!50100 */ format, but doesn't ignore specified partition names + tk.MustExec("CREATE TABLE tkey11 (`col1` int, `col2` char(5),`col3` date)" + + "/*!50100 PARTITION BY KEY (col1) PARTITIONS 4 \n" + + "(PARTITION `pp0`,\n" + + "PARTITION `pp1`,\n" + + "PARTITION `pp2`,\n" + + "PARTITION `pp3`)\n" + + "*/") + result = tk.MustQuery("show create table tkey11") + result.Check(testkit.Rows("tkey11 CREATE TABLE `tkey11` (\n" + + " `col1` int(11) DEFAULT NULL,\n" + + " `col2` char(5) DEFAULT NULL,\n" + + " `col3` date DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY KEY (`col1`)\n" + + "(PARTITION `pp0`,\n" + + " PARTITION `pp1`,\n" + + " PARTITION `pp2`,\n" + + " PARTITION `pp3`)")) + + // It shows the comment defined in the ddl + tk.MustExec("CREATE TABLE tkey12 (`col1` int, `col2` char(5),`col3` date)" + + "PARTITION BY KEY (col1) \n" + + "(PARTITION `pp0` comment 'huaian',\n" + + "PARTITION `pp1` comment 'nanjing',\n" + + "PARTITION `pp2` comment 'zhenjiang',\n" + + "PARTITION `pp3` comment 'suzhou')\n") + result = tk.MustQuery("show create table tkey12") + result.Check(testkit.Rows("tkey12 CREATE TABLE `tkey12` (\n" + + " `col1` int(11) DEFAULT NULL,\n" + + " `col2` char(5) DEFAULT NULL,\n" + + " `col3` date DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY KEY (`col1`)\n" + + "(PARTITION `pp0` COMMENT 'huaian',\n" + + " PARTITION `pp1` COMMENT 'nanjing',\n" + + " PARTITION `pp2` COMMENT 'zhenjiang',\n" + + " PARTITION `pp3` COMMENT 'suzhou')")) + + // It shows the placement policy defined in the ddl + tk.MustExec("drop placement policy if exists fivereplicas") + tk.MustExec("CREATE PLACEMENT POLICY fivereplicas FOLLOWERS=4") + tk.MustExec("CREATE TABLE tkey13 (`col1` int, `col2` char(5),`col3` date) placement policy fivereplicas\n" + + "PARTITION BY KEY (col1) PARTITIONS 4") + result = tk.MustQuery("show create table tkey13") + result.Check(testkit.Rows("tkey13 CREATE TABLE `tkey13` (\n" + + " `col1` int(11) DEFAULT NULL,\n" + + " `col2` char(5) DEFAULT NULL,\n" + + " `col3` date DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`fivereplicas` */\n" + + "PARTITION BY KEY (`col1`) PARTITIONS 4")) + + // The partition column can has null value + tk.MustExec("CREATE TABLE tkey14 (`col1` int, `col2` int,`col3` int, col4 int)\n" + + "PARTITION BY KEY (col3) PARTITIONS 4") + tk.MustExec("INSERT INTO tkey14 values(20,1,1,1),(1,2,NULL,2),(3,3,3,3),(3,3,NULL,3),(4,4,4,4),(5,5,5,5),(6,6,null,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,null,14)") + tk.MustQuery("SELECT count(*) FROM tkey14 WHERE col3 = NULL").Check(testkit.Rows("0")) + tk.MustQuery("SELECT count(*) FROM tkey14 WHERE col3 IS NULL").Check(testkit.Rows("4")) + result = tk.MustQuery("EXPLAIN SELECT count(*) FROM tkey14 WHERE col3 IS NULL") + result.CheckContain("partition:p1") + result.MultiCheckNotContain([]string{"partition:p0", "partition:p2", "partition:p3"}) + + tk.MustExec("CREATE TABLE tkey15 (`col1` int, col2 DATE NOT NULL,col3 VARCHAR(12), col4 int)\n" + + "PARTITION BY KEY (col3) PARTITIONS 4") + tk.MustExec("INSERT INTO tkey15 VALUES(1, '2023-02-22', 'linpin', 1), (2, '2023-02-22', NULL, 2), (3, '2023-02-22', 'anqila', 3), (4, '2023-02-22', NULL, 4)") + result = tk.MustQuery("EXPLAIN SELECT count(*) FROM tkey15 WHERE col3 IS NULL") + result.CheckContain("partition:p1") + result.MultiCheckNotContain([]string{"partition:p0", "partition:p2", "partition:p3"}) + + tk.MustExec("CREATE TABLE tkey12_2 (col1 INT, col2 INT ,col3 INT ,col4 INT , UNIQUE KEY(col2, col3)" + + ") PARTITION BY KEY(col2, col3) PARTITIONS 4") + tk.MustExec("INSERT INTO tkey12_2 values(20,1,1,1),(1,2,NULL,2),(3,3,3,3),(3,3,NULL,3),(4,4,4,4)," + + "(5,5,5,5), (6,6,null,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,null,14)") + result = tk.MustQuery("EXPLAIN SELECT * FROM tkey12_2 WHERE col2 = 2 and col3 IS NULL") + result.MultiCheckNotContain([]string{"partition:p1", "partition:p0", "partition:p3"}) + tk.MustQuery("SELECT * FROM tkey12_2 WHERE col2 = 2 and col3 IS NULL").Check(testkit.Rows("1 2 2")) + result = tk.MustQuery("EXPLAIN SELECT * FROM tkey12_2 WHERE col2 = 2") + result.MultiCheckContain([]string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}) + tk.MustQuery("SELECT * FROM tkey12_2 WHERE col2 = 2").Check(testkit.Rows("1 2 2")) + tk.MustQuery("EXPLAIN SELECT * FROM tkey12_2 WHERE col2 = 2").MultiCheckContain([]string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}) + tk.MustQuery("SELECT * FROM tkey12_2 WHERE col2 IS NULL") + tk.MustQuery("EXPLAIN SELECT * FROM tkey12_2 WHERE col2 IS NULL").MultiCheckContain([]string{"partition:p0", "partition:p1", "partition:p2", "partition:p3"}) + // Get the partition information from information_schema.partitions + result = tk.MustQuery("select PARTITION_NAME,PARTITION_ORDINAL_POSITION,PARTITION_METHOD,PARTITION_EXPRESSION " + + "FROM information_schema.partitions where TABLE_NAME = 'tkey12_2'") + result.Check(testkit.Rows("p0 1 KEY `col2`,`col3`", "p1 2 KEY `col2`,`col3`", "p2 3 KEY `col2`,`col3`", "p3 4 KEY `col2`,`col3`")) + + // This tests caculating the boundary partition ID when it prunes partition table + tk.MustExec("create table tkey16 (a int) partition by key (a) partitions 12") + tk.MustExec("insert into tkey16 values (0), (1), (2), (3)") + tk.MustExec("insert into tkey16 select a + 4 from tkey16") + tk.MustExec("insert into tkey16 select a + 8 from tkey16") + tk.MustExec("select * from information_schema.partitions where partition_name is not null") +} + +func TestKeyPartitionWithDifferentCharsets(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database partitiondb4") + defer tk.MustExec("drop database partitiondb4") + tk.MustExec("use partitiondb4") + + tk.MustExec("CREATE TABLE tkey29 (" + + "col1 INT NOT NULL," + + "col2 DATE NOT NULL," + + "col3 VARCHAR(12) NOT NULL," + + "col4 INT NOT NULL," + + "UNIQUE KEY (col3)" + + ") CHARSET=utf8mb4 COLLATE=utf8mb4_bin " + + "PARTITION BY KEY(col3) " + + "PARTITIONS 4") + // ignore tail spaces + err := tk.ExecToErr("INSERT INTO tkey29 VALUES(1, '2023-02-22', 'linpin', 1), (1, '2023-02-22', 'linpin ', 5)") + require.Regexp(t, "Duplicate entry 'linpin ' for key 'tkey29.col3'", err) + // case sensitive + tk.MustExec("INSERT INTO tkey29 VALUES(3, '2023-02-22', 'abc', 1), (4, '2023-02-22', 'ABC ', 5)") + + tk.MustExec("CREATE TABLE tkey30 (" + + "col1 INT NOT NULL," + + "col2 DATE NOT NULL," + + "col3 VARCHAR(12) NOT NULL," + + "col4 INT NOT NULL," + + "UNIQUE KEY (col3)" + + ") CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci " + + "PARTITION BY KEY(col3) " + + "PARTITIONS 4") + // case insensitive + err = tk.ExecToErr("INSERT INTO tkey30 VALUES(1, '2023-02-22', 'linpin', 1), (1, '2023-02-22', 'LINPIN', 5)") + require.Regexp(t, "Duplicate entry 'LINPIN' for key 'tkey30.col3'", err) + // ignore tail spaces + err = tk.ExecToErr("INSERT INTO tkey30 VALUES(1, '2023-02-22', 'linpin', 1), (1, '2023-02-22', 'LINPIN ', 5)") + require.Regexp(t, "Duplicate entry 'LINPIN ' for key 'tkey30.col3'", err) + + tk.MustExec("CREATE TABLE tkey31 (" + + "col1 INT NOT NULL," + + "col2 DATE NOT NULL," + + "col3 VARCHAR(12) NOT NULL," + + "col4 INT NOT NULL," + + "UNIQUE KEY (col3)" + + ") CHARSET=gbk COLLATE=gbk_chinese_ci " + + "PARTITION BY KEY(col3) " + + "PARTITIONS 4") + err = tk.ExecToErr("INSERT INTO tkey31 VALUES(1, '2023-02-22', '刘德华', 1), (1, '2023-02-22', '刘德华 ', 5)") + require.Regexp(t, "Duplicate entry '刘德华 ' for key 'tkey31.col3'", err) + tk.MustExec("INSERT INTO tkey31 VALUES(1, '2023-02-22', '刘德华', 1), (5, '2023-02-22', '张学友', 5),(6, '2023-02-22', '艾伦', 6), (7, '2023-02-22', '宁采臣', 7)") + tk.MustQuery("SELECT * FROM tkey31 partition(p0)").Check(testkit.Rows("1 2023-02-22 刘德华 1")) + tk.MustQuery("SELECT * FROM tkey31 partition(p1)").Check(testkit.Rows("6 2023-02-22 艾伦 6")) + tk.MustQuery("SELECT * FROM tkey31 partition(p2)").Check(testkit.Rows("5 2023-02-22 张学友 5")) + tk.MustQuery("SELECT * FROM tkey31 partition(p3)").Check(testkit.Rows("7 2023-02-22 宁采臣 7")) +} + func TestIssue31721(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -714,6 +2261,82 @@ func TestIssue31721(t *testing.T) { tk.MustExec("select * from t_31721 partition(p0, p1) where col1 != 2;") } +func TestKeyPartitionTableDDL(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database partitiondb3") + defer tk.MustExec("drop database partitiondb3") + tk.MustExec("use partitiondb3") + tk.MustExec("CREATE TABLE tkey14 (\n" + + "col1 INT NOT NULL," + + "col2 INT NOT NULL," + + "col3 INT NOT NULL," + + "col4 INT NOT NULL," + + "primary KEY (col1,col3)\n" + + ")" + + "PARTITION BY KEY(col3) PARTITIONS 4") + tk.MustExec("INSERT INTO tkey14 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)") + + tk.MustExec("CREATE TABLE tkey15 (\n" + + "col1 INT NOT NULL," + + "col2 INT NOT NULL," + + "col3 INT NOT NULL," + + "col4 INT NOT NULL," + + "primary KEY (col1,col3)\n" + + ")") + tk.MustExec("INSERT INTO tkey15 values (20,20,20,20)") + + tk.MustExec("CREATE TABLE tkey16 (\n" + + "col1 INT NOT NULL," + + "col2 INT NOT NULL," + + "col3 INT NOT NULL," + + "col4 INT NOT NULL," + + "primary KEY (col1,col3)\n" + + ")" + + "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) + err = tk.ExecToErr("ALTER TABLE tkey14 ADD PARTITION PARTITIONS 1") + require.Regexp(t, "Unsupported add partitions", err) + 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")) + err = tk.ExecToErr("ALTER TABLE tkey16 COALESCE PARTITION 2") + require.Regexp(t, "Unsupported coalesce partitions", err) + tk.MustExec("ALTER TABLE tkey14 ANALYZE PARTITION p3") + err = tk.ExecToErr("ALTER TABLE tkey14 CHECK PARTITION p2") + require.Regexp(t, "Unsupported check partition", err) + err = tk.ExecToErr("ALTER TABLE tkey14 OPTIMIZE PARTITION p2") + require.Regexp(t, "Unsupported optimize partition", err) + err = tk.ExecToErr("ALTER TABLE tkey14 REBUILD PARTITION p2") + require.Regexp(t, "Unsupported rebuild partition", err) + err = tk.ExecToErr("ALTER TABLE tkey14 EXCHANGE PARTITION p3 WITH TABLE tkey15") + require.Regexp(t, "Unsupported partition type of table tkey14 when exchanging partition", err) + + err = tk.ExecToErr("ALTER TABLE tkey16 REORGANIZE PARTITION") + require.Regexp(t, "Unsupported reorganize partition", err) + err = tk.ExecToErr("ALTER TABLE tkey16 REORGANIZE PARTITION p0 INTO (PARTITION p0,PARTITION p1)") + require.Regexp(t, "Unsupported reorganize partition", err) + err = tk.ExecToErr("ALTER TABLE tkey16 REORGANIZE PARTITION p0 INTO (PARTITION p0)") + 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("CREATE TABLE tkey17 (" + + "id INT NOT NULL PRIMARY KEY," + + "name VARCHAR(20)" + + ")" + + "PARTITION BY KEY()" + + "PARTITIONS 2") + result := tk.MustQuery("show warnings") + result.CheckContain("Unsupported partition type KEY, treat as normal table") +} + func TestPruneModeWarningInfo(t *testing.T) { store := testkit.CreateMockStore(t) diff --git a/testkit/result.go b/testkit/result.go index 210d32d4c57b9..b4a49f2403c5c 100644 --- a/testkit/result.go +++ b/testkit/result.go @@ -130,3 +130,42 @@ func (res *Result) CheckAt(cols []int, expected [][]interface{}) { need := fmt.Sprintf("%s", expected) res.require.Equal(need, got, res.comment) } + +// CheckContain checks whether the result contains the expected string +func (res *Result) CheckContain(expected string) { + for _, row := range res.rows { + for _, colValue := range row { + if strings.Contains(colValue, expected) { + return + } + } + } + comment := fmt.Sprintf("the result doesn't contain the exepected %s", expected) + res.require.Equal(true, false, comment) +} + +// MultiCheckContain checks whether the result contains strings in `expecteds` +func (res *Result) MultiCheckContain(expecteds []string) { + for _, expected := range expecteds { + res.CheckContain(expected) + } +} + +// CheckNotContain checks whether the result doesn't contain the expected string +func (res *Result) CheckNotContain(unexpected string) { + for _, row := range res.rows { + for _, colValue := range row { + if strings.Contains(colValue, unexpected) { + comment := fmt.Sprintf("the result contain the unexepected %s", unexpected) + res.require.Equal(true, false, comment) + } + } + } +} + +// MultiCheckNotContain checks whether the result doesn't contain the strings in `expected` +func (res *Result) MultiCheckNotContain(unexpecteds []string) { + for _, unexpected := range unexpecteds { + res.CheckNotContain(unexpected) + } +} diff --git a/ttl/ttlworker/BUILD.bazel b/ttl/ttlworker/BUILD.bazel index 943f206ec16fe..aa805ad1a1920 100644 --- a/ttl/ttlworker/BUILD.bazel +++ b/ttl/ttlworker/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//parser/terror", "//sessionctx", "//sessionctx/variable", + "//store/driver/error", "//ttl/cache", "//ttl/client", "//ttl/metrics", @@ -35,6 +36,7 @@ go_library( "@com_github_ngaut_pools//:pools", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", + "@com_github_tikv_client_go_v2//tikv", "@io_etcd_go_etcd_client_v3//:client", "@org_golang_x_time//rate", "@org_uber_go_multierr//:multierr", @@ -69,6 +71,7 @@ go_test( "//sessionctx", "//sessionctx/variable", "//statistics/handle", + "//store/mockstore", "//testkit", "//ttl/cache", "//ttl/client", @@ -83,6 +86,7 @@ go_test( "@com_github_prometheus_client_model//go", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//testutils", "@org_golang_x_time//rate", "@org_uber_go_atomic//:atomic", "@org_uber_go_zap//:zap", diff --git a/ttl/ttlworker/config.go b/ttl/ttlworker/config.go index 89ca9eedae010..b5cef0b0ecc11 100644 --- a/ttl/ttlworker/config.go +++ b/ttl/ttlworker/config.go @@ -61,3 +61,10 @@ func getTaskManagerLoopTickerInterval() time.Duration { }) return taskManagerLoopTickerInterval } + +func getTaskManagerHeartBeatExpireInterval() time.Duration { + failpoint.Inject("task-manager-heartbeat-expire-interval", func(val failpoint.Value) time.Duration { + return time.Duration(val.(int)) + }) + return 2 * ttlTaskHeartBeatTickerInterval +} diff --git a/ttl/ttlworker/job_manager.go b/ttl/ttlworker/job_manager.go index f9470ead93622..5c0edce652d9f 100644 --- a/ttl/ttlworker/job_manager.go +++ b/ttl/ttlworker/job_manager.go @@ -125,7 +125,7 @@ func NewJobManager(id string, sessPool sessionPool, store kv.Storage, etcdCli *c manager.notificationCli = client.NewMockNotificationClient() } - manager.taskManager = newTaskManager(manager.ctx, sessPool, manager.infoSchemaCache, id) + manager.taskManager = newTaskManager(manager.ctx, sessPool, manager.infoSchemaCache, id, store) return } diff --git a/ttl/ttlworker/task_manager.go b/ttl/ttlworker/task_manager.go index 4abd2d0098924..780c16d09425a 100644 --- a/ttl/ttlworker/task_manager.go +++ b/ttl/ttlworker/task_manager.go @@ -20,11 +20,14 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/variable" + storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/ttl/cache" "github.com/pingcap/tidb/ttl/metrics" "github.com/pingcap/tidb/ttl/session" "github.com/pingcap/tidb/util/logutil" + "github.com/tikv/client-go/v2/tikv" "go.uber.org/multierr" "go.uber.org/zap" ) @@ -67,6 +70,11 @@ func updateTTLTaskHeartBeatSQL(jobID string, scanID int64, now time.Time, state return updateTTLTaskHeartBeatTempalte, []interface{}{string(stateStr), now.Format(timeFormat), jobID, scanID}, nil } +const countRunningTasks = "SELECT count(1) FROM mysql.tidb_ttl_task WHERE status = 'running'" + +var errAlreadyScheduled = errors.New("task is already scheduled") +var errTooManyRunningTasks = errors.New("there are too many running tasks") + // taskManager schedules and manages the ttl tasks on this instance type taskManager struct { ctx context.Context @@ -74,6 +82,8 @@ type taskManager struct { id string + store kv.Storage + scanWorkers []worker delWorkers []worker @@ -84,13 +94,15 @@ type taskManager struct { notifyStateCh chan interface{} } -func newTaskManager(ctx context.Context, sessPool sessionPool, infoSchemaCache *cache.InfoSchemaCache, id string) *taskManager { +func newTaskManager(ctx context.Context, sessPool sessionPool, infoSchemaCache *cache.InfoSchemaCache, id string, store kv.Storage) *taskManager { return &taskManager{ ctx: logutil.WithKeyValue(ctx, "ttl-worker", "task-manager"), sessPool: sessPool, id: id, + store: store, + scanWorkers: []worker{}, delWorkers: []worker{}, @@ -269,14 +281,22 @@ func (m *taskManager) rescheduleTasks(se session.Session, now time.Time) { return } +loop: for _, t := range tasks { logger := logutil.Logger(m.ctx).With(zap.String("jobID", t.JobID), zap.Int64("scanID", t.ScanID)) task, err := m.lockScanTask(se, t, now) if err != nil { - // If other nodes lock the task, it will return an error. It's expected - // so the log level is only `info` - logutil.Logger(m.ctx).Info("fail to lock scan task", zap.Error(err)) + switch errors.Cause(err) { + case errAlreadyScheduled: + continue + case errTooManyRunningTasks: + break loop + case storeerr.ErrLockWaitTimeout: + // don't step into the next step to avoid exceeding the limit + break loop + } + logutil.Logger(m.ctx).Warn("fail to lock scan task", zap.Error(err)) continue } @@ -299,7 +319,7 @@ func (m *taskManager) rescheduleTasks(se session.Session, now time.Time) { } func (m *taskManager) peekWaitingScanTasks(se session.Session, now time.Time) ([]*cache.TTLTask, error) { - sql, args := cache.PeekWaitingTTLTask(now.Add(-2 * ttlTaskHeartBeatTickerInterval)) + sql, args := cache.PeekWaitingTTLTask(now.Add(-getTaskManagerHeartBeatExpireInterval())) rows, err := se.ExecuteSQL(m.ctx, sql, args...) if err != nil { return nil, errors.Wrapf(err, "execute sql: %s", sql) @@ -328,12 +348,26 @@ func (m *taskManager) lockScanTask(se session.Session, task *cache.TTLTask, now err := se.RunInTxn(ctx, func() error { var err error + // The `for update` here ensures two things: + // 1. The instance which will update this row has committed, and we can get the newest information. + // 2. There is only one successful transaction concurrently. + // If it's a `for update nowait`, we cannot avoid the situation that multiple transactions fetched the tasks + // at the same time, and the task limitation doesn't work. task, err = m.syncTaskFromTable(se, task.JobID, task.ScanID, true) if err != nil { return err } - if task.OwnerID != "" && !task.OwnerHBTime.Add(2*jobManagerLoopTickerInterval).Before(now) { - return errors.New("task is already scheduled") + if task.OwnerID != "" && !task.OwnerHBTime.Add(getTaskManagerHeartBeatExpireInterval()).Before(now) { + return errors.WithStack(errAlreadyScheduled) + } + + // check the count of running task is smaller or equal than the limit + rows, err := se.ExecuteSQL(ctx, countRunningTasks) + if err != nil { + return errors.Wrapf(err, "execute sql: %s", countRunningTasks) + } + if !m.meetTTLRunningTask(int(rows[0].GetInt64(0))) { + return errors.WithStack(errTooManyRunningTasks) } sql, args := setTTLTaskOwnerSQL(task.JobID, task.ScanID, m.id, now) @@ -370,12 +404,12 @@ func (m *taskManager) lockScanTask(se session.Session, task *cache.TTLTask, now }, nil } -func (m *taskManager) syncTaskFromTable(se session.Session, jobID string, scanID int64, detectLock bool) (*cache.TTLTask, error) { +func (m *taskManager) syncTaskFromTable(se session.Session, jobID string, scanID int64, waitLock bool) (*cache.TTLTask, error) { ctx := m.ctx sql, args := cache.SelectFromTTLTaskWithID(jobID, scanID) - if detectLock { - sql += " FOR UPDATE NOWAIT" + if waitLock { + sql += " FOR UPDATE" } rows, err := se.ExecuteSQL(ctx, sql, args...) if err != nil { @@ -510,6 +544,30 @@ func (m *taskManager) reportMetrics() { metrics.DeletingTaskCnt.Set(float64(deletingTaskCnt)) } +func (m *taskManager) meetTTLRunningTask(count int) bool { + ttlRunningTask := variable.TTLRunningTasks.Load() + // `-1` is the auto value, means we should calculate the limit according to the count of TiKV + if ttlRunningTask != -1 { + return int(ttlRunningTask) > count + } + + store, ok := m.store.(tikv.Storage) + if !ok { + return variable.MaxConfigurableConcurrency > count + } + + regionCache := store.GetRegionCache() + if regionCache == nil { + return true + } + limit := len(regionCache.GetAllStores()) + if limit > variable.MaxConfigurableConcurrency { + limit = variable.MaxConfigurableConcurrency + } + + return limit > count +} + type runningScanTask struct { *ttlScanTask cancel func() diff --git a/ttl/ttlworker/task_manager_integration_test.go b/ttl/ttlworker/task_manager_integration_test.go index 19bf25fd3b1d6..30d2742657591 100644 --- a/ttl/ttlworker/task_manager_integration_test.go +++ b/ttl/ttlworker/task_manager_integration_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/ttl/cache" "github.com/pingcap/tidb/ttl/metrics" @@ -31,6 +32,7 @@ import ( "github.com/pingcap/tidb/util/logutil" dto "github.com/prometheus/client_model/go" "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/testutils" "go.uber.org/atomic" "go.uber.org/zap" ) @@ -38,6 +40,7 @@ import ( func TestParallelLockNewTask(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_ttl_running_tasks = 1000") ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnTTL) tk.MustExec("create table test.t (id int, created_at datetime) TTL= created_at + interval 1 hour") testTable, err := tk.Session().GetDomainInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("t")) @@ -50,7 +53,7 @@ func TestParallelLockNewTask(t *testing.T) { isc := cache.NewInfoSchemaCache(time.Minute) require.NoError(t, isc.Update(se)) - m := ttlworker.NewTaskManager(context.Background(), nil, isc, "test-id") + m := ttlworker.NewTaskManager(context.Background(), nil, isc, "test-id", store) // insert and lock a new task sql, args, err := cache.InsertIntoTTLTask(tk.Session(), "test-job", testTable.Meta().ID, 1, nil, nil, now, now) @@ -87,7 +90,7 @@ func TestParallelLockNewTask(t *testing.T) { isc := cache.NewInfoSchemaCache(time.Minute) require.NoError(t, isc.Update(se)) - m := ttlworker.NewTaskManager(context.Background(), nil, isc, scanManagerID) + m := ttlworker.NewTaskManager(context.Background(), nil, isc, scanManagerID, store) _, err := m.LockScanTask(se, &cache.TTLTask{ ScanID: 1, @@ -113,6 +116,7 @@ func TestParallelSchedule(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) waitAndStopTTLManager(t, dom) tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_ttl_running_tasks = 1000") sessionFactory := sessionFactory(t, store) tk.MustExec("create table test.t(id int, created_at datetime) ttl=created_at + interval 1 day") @@ -135,7 +139,7 @@ func TestParallelSchedule(t *testing.T) { workers = append(workers, scanWorker) } - m := ttlworker.NewTaskManager(context.Background(), nil, isc, fmt.Sprintf("task-manager-%d", i)) + m := ttlworker.NewTaskManager(context.Background(), nil, isc, fmt.Sprintf("task-manager-%d", i), store) m.SetScanWorkers4Test(workers) scheduleWg.Add(1) go func() { @@ -157,6 +161,7 @@ func TestTaskScheduleExpireHeartBeat(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) waitAndStopTTLManager(t, dom) tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_ttl_running_tasks = 1000") sessionFactory := sessionFactory(t, store) // create table and scan task @@ -174,7 +179,7 @@ func TestTaskScheduleExpireHeartBeat(t *testing.T) { // schedule in a task manager scanWorker := ttlworker.NewMockScanWorker(t) scanWorker.Start() - m := ttlworker.NewTaskManager(context.Background(), nil, isc, "task-manager-1") + m := ttlworker.NewTaskManager(context.Background(), nil, isc, "task-manager-1", store) m.SetScanWorkers4Test([]ttlworker.Worker{scanWorker}) m.RescheduleTasks(sessionFactory(), now) tk.MustQuery("select status,owner_id from mysql.tidb_ttl_task").Check(testkit.Rows("running task-manager-1")) @@ -182,7 +187,7 @@ func TestTaskScheduleExpireHeartBeat(t *testing.T) { // another task manager should fetch this task after heartbeat expire scanWorker2 := ttlworker.NewMockScanWorker(t) scanWorker2.Start() - m2 := ttlworker.NewTaskManager(context.Background(), nil, isc, "task-manager-2") + m2 := ttlworker.NewTaskManager(context.Background(), nil, isc, "task-manager-2", store) m2.SetScanWorkers4Test([]ttlworker.Worker{scanWorker2}) m2.RescheduleTasks(sessionFactory(), now.Add(time.Hour)) tk.MustQuery("select status,owner_id from mysql.tidb_ttl_task").Check(testkit.Rows("running task-manager-2")) @@ -193,7 +198,7 @@ func TestTaskScheduleExpireHeartBeat(t *testing.T) { m2.CheckFinishedTask(sessionFactory(), now) scanWorker3 := ttlworker.NewMockScanWorker(t) scanWorker3.Start() - m3 := ttlworker.NewTaskManager(context.Background(), nil, isc, "task-manager-3") + m3 := ttlworker.NewTaskManager(context.Background(), nil, isc, "task-manager-3", store) m3.SetScanWorkers4Test([]ttlworker.Worker{scanWorker3}) m3.RescheduleTasks(sessionFactory(), now.Add(time.Hour)) tk.MustQuery("select status,owner_id from mysql.tidb_ttl_task").Check(testkit.Rows("finished task-manager-2")) @@ -203,6 +208,7 @@ func TestTaskMetrics(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) waitAndStopTTLManager(t, dom) tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_ttl_running_tasks = 1000") sessionFactory := sessionFactory(t, store) // create table and scan task @@ -220,7 +226,7 @@ func TestTaskMetrics(t *testing.T) { // schedule in a task manager scanWorker := ttlworker.NewMockScanWorker(t) scanWorker.Start() - m := ttlworker.NewTaskManager(context.Background(), nil, isc, "task-manager-1") + m := ttlworker.NewTaskManager(context.Background(), nil, isc, "task-manager-1", store) m.SetScanWorkers4Test([]ttlworker.Worker{scanWorker}) m.RescheduleTasks(sessionFactory(), now) tk.MustQuery("select status,owner_id from mysql.tidb_ttl_task").Check(testkit.Rows("running task-manager-1")) @@ -235,6 +241,7 @@ func TestRescheduleWithError(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) waitAndStopTTLManager(t, dom) tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_ttl_running_tasks = 1000") sessionFactory := sessionFactory(t, store) // insert a wrong scan task with random table id @@ -248,7 +255,7 @@ func TestRescheduleWithError(t *testing.T) { // schedule in a task manager scanWorker := ttlworker.NewMockScanWorker(t) scanWorker.Start() - m := ttlworker.NewTaskManager(context.Background(), nil, isc, "task-manager-1") + m := ttlworker.NewTaskManager(context.Background(), nil, isc, "task-manager-1", store) m.SetScanWorkers4Test([]ttlworker.Worker{scanWorker}) notify := make(chan struct{}) go func() { @@ -265,3 +272,69 @@ func TestRescheduleWithError(t *testing.T) { } tk.MustQuery("select status from mysql.tidb_ttl_task").Check(testkit.Rows("waiting")) } + +func TestTTLRunningTasksLimitation(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + waitAndStopTTLManager(t, dom) + tk := testkit.NewTestKit(t, store) + sessionFactory := sessionFactory(t, store) + + tk.MustExec("set global tidb_ttl_running_tasks = 32") + tk.MustExec("create table test.t(id int, created_at datetime) ttl=created_at + interval 1 day") + table, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + // 64 tasks and 128 scan workers (in 16 task manager) should only schedule 32 tasks + for i := 0; i < 128; i++ { + sql := fmt.Sprintf("insert into mysql.tidb_ttl_task(job_id,table_id,scan_id,expire_time,created_time) values ('test-job', %d, %d, NOW(), NOW())", table.Meta().ID, i) + tk.MustExec(sql) + } + isc := cache.NewInfoSchemaCache(time.Second) + require.NoError(t, isc.Update(sessionFactory())) + now := time.Now() + scheduleWg := sync.WaitGroup{} + for i := 0; i < 16; i++ { + workers := []ttlworker.Worker{} + for j := 0; j < 8; j++ { + scanWorker := ttlworker.NewMockScanWorker(t) + scanWorker.Start() + workers = append(workers, scanWorker) + } + + ctx := logutil.WithKeyValue(context.Background(), "ttl-worker-test", fmt.Sprintf("task-manager-%d", i)) + m := ttlworker.NewTaskManager(ctx, nil, isc, fmt.Sprintf("task-manager-%d", i), store) + m.SetScanWorkers4Test(workers) + scheduleWg.Add(1) + go func() { + se := sessionFactory() + m.RescheduleTasks(se, now) + scheduleWg.Done() + }() + } + scheduleWg.Wait() + // all tasks should have been scheduled + tk.MustQuery("select count(1) from mysql.tidb_ttl_task where status = 'running'").Check(testkit.Rows("32")) +} + +func TestMeetTTLRunningTasks(t *testing.T) { + // initialize a cluster with 3 TiKV + store, dom := testkit.CreateMockStoreAndDomain(t, mockstore.WithStoreType(mockstore.MockTiKV), + mockstore.WithClusterInspector(func(c testutils.Cluster) { + mockstore.BootstrapWithMultiStores(c, 3) + })) + waitAndStopTTLManager(t, dom) + tk := testkit.NewTestKit(t, store) + + // -1, the default value, means the count of TiKV + require.True(t, dom.TTLJobManager().TaskManager().MeetTTLRunningTasks(2)) + require.False(t, dom.TTLJobManager().TaskManager().MeetTTLRunningTasks(3)) + + // positive number means the limitation + tk.MustExec("set global tidb_ttl_running_tasks = 32") + require.False(t, dom.TTLJobManager().TaskManager().MeetTTLRunningTasks(32)) + require.True(t, dom.TTLJobManager().TaskManager().MeetTTLRunningTasks(31)) + + // set it back to auto value + tk.MustExec("set global tidb_ttl_running_tasks = -1") + require.True(t, dom.TTLJobManager().TaskManager().MeetTTLRunningTasks(2)) + require.False(t, dom.TTLJobManager().TaskManager().MeetTTLRunningTasks(3)) +} diff --git a/ttl/ttlworker/task_manager_test.go b/ttl/ttlworker/task_manager_test.go index cffb8e071b62d..6212aee17d0ce 100644 --- a/ttl/ttlworker/task_manager_test.go +++ b/ttl/ttlworker/task_manager_test.go @@ -64,6 +64,11 @@ func (m *taskManager) GetRunningTasks() []*runningScanTask { return m.runningTasks } +// MeetTTLRunningTasks is an exported version of meetTTLRunningTask +func (m *taskManager) MeetTTLRunningTasks(count int) bool { + return m.meetTTLRunningTask(count) +} + // ReportTaskFinished is an exported version of reportTaskFinished func (t *runningScanTask) SetResult(err error) { t.result = &ttlScanTaskExecResult{ @@ -80,7 +85,7 @@ func TestResizeWorkers(t *testing.T) { scanWorker1.Start() scanWorker2 := NewMockScanWorker(t) - m := newTaskManager(context.Background(), nil, nil, "test-id") + m := newTaskManager(context.Background(), nil, nil, "test-id", nil) m.sessPool = newMockSessionPool(t, tbl) m.SetScanWorkers4Test([]worker{ scanWorker1, @@ -99,7 +104,7 @@ func TestResizeWorkers(t *testing.T) { scanWorker2 = NewMockScanWorker(t) scanWorker2.Start() - m = newTaskManager(context.Background(), nil, nil, "test-id") + m = newTaskManager(context.Background(), nil, nil, "test-id", nil) m.sessPool = newMockSessionPool(t, tbl) m.SetScanWorkers4Test([]worker{ scanWorker1, @@ -115,7 +120,7 @@ func TestResizeWorkers(t *testing.T) { scanWorker2 = NewMockScanWorker(t) scanWorker2.Start() - m = newTaskManager(context.Background(), nil, nil, "test-id") + m = newTaskManager(context.Background(), nil, nil, "test-id", nil) m.sessPool = newMockSessionPool(t, tbl) m.SetScanWorkers4Test([]worker{ scanWorker1, diff --git a/types/convert.go b/types/convert.go index 715f49b140c0c..862997c2319a1 100644 --- a/types/convert.go +++ b/types/convert.go @@ -254,7 +254,7 @@ func convertDecimalStrToUint(sc *stmtctx.StatementContext, str string, upperBoun if intStr == "" { intStr = "0" } - if sc.ShouldClipToZero() && intStr[0] == '-' { + if intStr[0] == '-' { return 0, overflow(str, tp) } @@ -263,8 +263,7 @@ func convertDecimalStrToUint(sc *stmtctx.StatementContext, str string, upperBoun round++ } - upperBound -= round - upperStr := strconv.FormatUint(upperBound, 10) + upperStr := strconv.FormatUint(upperBound-round, 10) if len(intStr) > len(upperStr) || (len(intStr) == len(upperStr) && intStr > upperStr) { return upperBound, overflow(str, tp) diff --git a/types/convert_test.go b/types/convert_test.go index b28560dfbdf75..db9cb2ba691fd 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -1275,8 +1275,9 @@ func TestConvertDecimalStrToUint(t *testing.T) { {"9223372036854775807.4999", 9223372036854775807, true}, {"18446744073709551614.55", 18446744073709551615, true}, {"18446744073709551615.344", 18446744073709551615, true}, - {"18446744073709551615.544", 0, false}, + {"18446744073709551615.544", 18446744073709551615, false}, {"-111.111", 0, false}, + {"-10000000000000000000.0", 0, false}, } for _, ca := range cases { result, err := convertDecimalStrToUint(&stmtctx.StatementContext{}, ca.input, math.MaxUint64, 0) @@ -1284,7 +1285,15 @@ func TestConvertDecimalStrToUint(t *testing.T) { require.Error(t, err) } else { require.NoError(t, err) - require.Equal(t, ca.result, result) } + require.Equal(t, ca.result, result, "input=%v", ca.input) } + + result, err := convertDecimalStrToUint(&stmtctx.StatementContext{}, "-99.0", math.MaxUint8, 0) + require.Error(t, err) + require.Equal(t, uint64(0), result) + + result, err = convertDecimalStrToUint(&stmtctx.StatementContext{}, "-100.0", math.MaxUint8, 0) + require.Error(t, err) + require.Equal(t, uint64(0), result) } diff --git a/types/datum.go b/types/datum.go index ec3514de7fd89..f74ffde249024 100644 --- a/types/datum.go +++ b/types/datum.go @@ -1994,6 +1994,20 @@ func (d *Datum) ToBytes() ([]byte, error) { } } +// ToHashKey gets the bytes representation of the datum considering collation. +func (d *Datum) ToHashKey() ([]byte, error) { + switch d.k { + case KindString, KindBytes: + return collate.GetCollator(d.Collation()).Key(d.GetString()), nil + default: + str, err := d.ToString() + if err != nil { + return nil, errors.Trace(err) + } + return collate.GetCollator(d.Collation()).Key(str), nil + } +} + // ToMysqlJSON is similar to convertToMysqlJSON, except the // latter parses from string, but the former uses it as primitive. func (d *Datum) ToMysqlJSON() (j BinaryJSON, err error) { diff --git a/util/stmtsummary/v2/tests/table_test.go b/util/stmtsummary/v2/tests/table_test.go index 4ad3c29389c96..830624a3f7d3d 100644 --- a/util/stmtsummary/v2/tests/table_test.go +++ b/util/stmtsummary/v2/tests/table_test.go @@ -456,6 +456,48 @@ func TestStmtSummaryHistoryTableOther(t *testing.T) { )) } +func TestPerformanceSchemaforNonPrepPlanCache(t *testing.T) { + setupStmtSummary() + defer closeStmtSummary() + + store := testkit.CreateMockStore(t) + tk := newTestKitWithRoot(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t (a int, key(a))`) + tk.MustExec("set global tidb_enable_stmt_summary = 0") + tk.MustExec("set global tidb_enable_stmt_summary = 1") + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + + tk.MustExec(`select * from t where a=1`) + tk.MustExec(`select * from t where a=1`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + + tk.MustQuery("select exec_count, digest_text, prepared, plan_in_cache, plan_cache_hits, query_sample_text " + + "from information_schema.statements_summary where digest_text='select * from `t` where `a` = ?'").Check(testkit.Rows( + "2 select * from `t` where `a` = ? 0 1 1 select * from t where a=1 [arguments: 1]")) + + tk.MustExec(`select * from t where a=2`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustExec(`select * from t where a=3`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + + // exec_count 2->4, plan_cache_hits 1->3 + tk.MustQuery("select exec_count, digest_text, prepared, plan_in_cache, plan_cache_hits, query_sample_text " + + "from information_schema.statements_summary where digest_text='select * from `t` where `a` = ?'").Check(testkit.Rows( + "4 select * from `t` where `a` = ? 0 1 3 select * from t where a=1 [arguments: 1]")) + + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=0`) + tk.MustExec(`select * from t where a=2`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + tk.MustExec(`select * from t where a=3`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + + // exec_count 4->6, plan_cache_hits 3->3 + tk.MustQuery("select exec_count, digest_text, prepared, plan_in_cache, plan_cache_hits, query_sample_text " + + "from information_schema.statements_summary where digest_text='select * from `t` where `a` = ?'").Check(testkit.Rows( + "6 select * from `t` where `a` = ? 0 0 3 select * from t where a=1 [arguments: 1]")) +} + func TestPerformanceSchemaforPlanCache(t *testing.T) { setupStmtSummary() defer closeStmtSummary()